diff --git a/.github/dependabot.yml b/.github/dependabot.yml index d135c26b02cc..9719e1a53392 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,6 +24,7 @@ updates: directory: "/" schedule: interval: "daily" + open-pull-requests-limit: 20 - package-ecosystem: "npm" directory: "/pinot-controller/src/main/resources" diff --git a/.github/workflows/build-multi-arch-pinot-docker-image.yml b/.github/workflows/build-multi-arch-pinot-docker-image.yml index 25460a77bf70..e3314bec9c12 100644 --- a/.github/workflows/build-multi-arch-pinot-docker-image.yml +++ b/.github/workflows/build-multi-arch-pinot-docker-image.yml @@ -64,7 +64,7 @@ jobs: needs: [ generate-build-info ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} @@ -72,7 +72,7 @@ jobs: name: Set up QEMU with: platforms: linux/${{ matrix.arch }} - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image @@ -91,13 +91,13 @@ jobs: needs: [ generate-build-info, build-pinot-docker-image ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Create Multi-Arch Manifest diff --git a/.github/workflows/build-pinot-base-docker-image.yml b/.github/workflows/build-pinot-base-docker-image.yml index 36162d49051f..e079fde1c304 100644 --- a/.github/workflows/build-pinot-base-docker-image.yml +++ b/.github/workflows/build-pinot-base-docker-image.yml @@ -32,13 +32,13 @@ jobs: openJdkDist: [ "amazoncorretto", "ms-openjdk" ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-pinot-docker-image.yml b/.github/workflows/build-pinot-docker-image.yml index c8652f2775c7..c21e8bbadd6e 100644 --- a/.github/workflows/build-pinot-docker-image.yml +++ b/.github/workflows/build-pinot-docker-image.yml @@ -47,13 +47,13 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-presto-docker-image.yml b/.github/workflows/build-presto-docker-image.yml index 6c76671a3e3f..d054074f1551 100644 --- a/.github/workflows/build-presto-docker-image.yml +++ b/.github/workflows/build-presto-docker-image.yml @@ -47,13 +47,13 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-superset-docker-image.yml b/.github/workflows/build-superset-docker-image.yml index 19b720905ac3..9cc362d7c502 100644 --- a/.github/workflows/build-superset-docker-image.yml +++ b/.github/workflows/build-superset-docker-image.yml @@ -43,13 +43,13 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/pinot_compatibility_tests.yml b/.github/workflows/pinot_compatibility_tests.yml index f2ff0f95f7f8..e9d242fede91 100644 --- a/.github/workflows/pinot_compatibility_tests.yml +++ b/.github/workflows/pinot_compatibility_tests.yml @@ -44,7 +44,7 @@ jobs: distribution: 'temurin' cache: 'maven' - name: Setup node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: v10.16.1 cache: 'npm' diff --git a/.github/workflows/pinot_tests.yml b/.github/workflows/pinot_tests.yml index a0c4b4861e10..1d90331719c4 100644 --- a/.github/workflows/pinot_tests.yml +++ b/.github/workflows/pinot_tests.yml @@ -271,7 +271,7 @@ jobs: ] name: Pinot Compatibility Regression Testing against ${{ matrix.old_commit }} on ${{ matrix.test_suite }} steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Set up JDK 11 uses: actions/setup-java@v4 with: @@ -279,7 +279,7 @@ jobs: distribution: 'temurin' cache: 'maven' - name: Setup node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: v10.16.1 cache: 'npm' diff --git a/.github/workflows/pinot_vuln_check.yml b/.github/workflows/pinot_vuln_check.yml index 4eebc1fb1074..b4924d3bc848 100644 --- a/.github/workflows/pinot_vuln_check.yml +++ b/.github/workflows/pinot_vuln_check.yml @@ -60,6 +60,6 @@ jobs: severity: 'CRITICAL,HIGH' timeout: '10m' - name: Upload Trivy scan results to GitHub Security tab - uses: github/codeql-action/upload-sarif@v2 + uses: github/codeql-action/upload-sarif@v3 with: sarif_file: 'trivy-results.sarif' diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 009c706a2258..177fc5d6f1e8 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -23,11 +23,11 @@ com.gradle gradle-enterprise-maven-extension - 1.19.2 + 1.20.1 com.gradle common-custom-user-data-maven-extension - 1.13 + 2 diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java index 31bae2215aac..b2d4e24d3f32 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java @@ -257,7 +257,7 @@ public boolean cancelQuery(long requestId, int timeoutMs, Executor executor, Htt @Override public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { requestContext.setRequestArrivalTimeMillis(System.currentTimeMillis()); @@ -307,7 +307,7 @@ public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOption protected BrokerResponse handleRequest(long requestId, String query, @Nullable SqlNodeAndOptions sqlNodeAndOptions, JsonNode request, @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, - HttpHeaders httpHeaders) + @Nullable HttpHeaders httpHeaders) throws Exception { LOGGER.debug("SQL query for request {}: {}", requestId, query); @@ -378,8 +378,9 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S boolean ignoreCase = _tableCache.isIgnoreCase(); String tableName; try { - tableName = getActualTableName( - DatabaseUtils.translateTableName(dataSource.getTableName(), httpHeaders, ignoreCase), _tableCache); + tableName = + getActualTableName(DatabaseUtils.translateTableName(dataSource.getTableName(), httpHeaders, ignoreCase), + _tableCache); } catch (DatabaseConflictException e) { LOGGER.info("{}. Request {}: {}", e.getMessage(), requestId, query); _brokerMetrics.addMeteredGlobalValue(BrokerMeter.QUERY_VALIDATION_EXCEPTIONS, 1); @@ -912,7 +913,7 @@ private String getServerTenant(String tableNameWithType) { *

Currently only supports subquery within the filter. */ private void handleSubquery(PinotQuery pinotQuery, long requestId, JsonNode jsonRequest, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { Expression filterExpression = pinotQuery.getFilterExpression(); if (filterExpression != null) { @@ -928,7 +929,7 @@ private void handleSubquery(PinotQuery pinotQuery, long requestId, JsonNode json * IN_ID_SET transform function. */ private void handleSubquery(Expression expression, long requestId, JsonNode jsonRequest, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { Function function = expression.getFunctionCall(); if (function == null) { diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java index 99df35a09ffa..4aa7b26eaab4 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java @@ -39,11 +39,11 @@ public interface BrokerRequestHandler { void shutDown(); BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception; default BrokerResponse handleRequest(JsonNode request, @Nullable RequesterIdentity requesterIdentity, - RequestContext requestContext, HttpHeaders httpHeaders) + RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { return handleRequest(request, null, requesterIdentity, requestContext, httpHeaders); } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java index fb87f715c185..0da84e6eadc4 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java @@ -82,7 +82,7 @@ public void shutDown() { @Override public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { requestContext.setBrokerId(_brokerId); if (sqlNodeAndOptions == null) { @@ -96,13 +96,13 @@ public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOption } } if (request.has(CommonConstants.Broker.Request.QUERY_OPTIONS)) { - sqlNodeAndOptions.setExtraOptions(RequestUtils.getOptionsFromJson(request, - CommonConstants.Broker.Request.QUERY_OPTIONS)); + sqlNodeAndOptions.setExtraOptions( + RequestUtils.getOptionsFromJson(request, CommonConstants.Broker.Request.QUERY_OPTIONS)); } - if (_multiStageBrokerRequestHandler != null && Boolean.parseBoolean(sqlNodeAndOptions.getOptions().get( - CommonConstants.Broker.Request.QueryOptionKey.USE_MULTISTAGE_ENGINE))) { - return _multiStageBrokerRequestHandler.handleRequest(request, requesterIdentity, requestContext, httpHeaders); + if (_multiStageBrokerRequestHandler != null && Boolean.parseBoolean( + sqlNodeAndOptions.getOptions().get(CommonConstants.Broker.Request.QueryOptionKey.USE_MULTISTAGE_ENGINE))) { + return _multiStageBrokerRequestHandler.handleRequest(request, requesterIdentity, requestContext, httpHeaders); } else { return _singleStageBrokerRequestHandler.handleRequest(request, sqlNodeAndOptions, requesterIdentity, requestContext, httpHeaders); diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index ee9f6cf19a74..01e4884d6a08 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -31,7 +31,6 @@ import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.broker.api.AccessControl; import org.apache.pinot.broker.api.RequesterIdentity; @@ -39,6 +38,7 @@ import org.apache.pinot.broker.querylog.QueryLogger; import org.apache.pinot.broker.queryquota.QueryQuotaManager; import org.apache.pinot.broker.routing.BrokerRoutingManager; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.metrics.BrokerMeter; diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java index 6135982e185c..423eb527faee 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java @@ -18,8 +18,6 @@ */ package org.apache.pinot.broker.routing.segmentpruner; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,7 +35,6 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.DateTimeFieldSpec; -import org.apache.pinot.spi.data.DateTimeFormatSpec; import org.apache.pinot.spi.data.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +65,7 @@ public static List getSegmentPruners(TableConfig tableConfig, List configuredSegmentPruners = new ArrayList<>(segmentPrunerTypes.size()); for (String segmentPrunerType : segmentPrunerTypes) { if (RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE.equalsIgnoreCase(segmentPrunerType)) { - SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig, propertyStore); + SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig); if (partitionSegmentPruner != null) { configuredSegmentPruners.add(partitionSegmentPruner); } @@ -91,7 +88,7 @@ public static List getSegmentPruners(TableConfig tableConfig, if ((tableType == TableType.OFFLINE && LEGACY_PARTITION_AWARE_OFFLINE_ROUTING.equalsIgnoreCase( routingTableBuilderName)) || (tableType == TableType.REALTIME && LEGACY_PARTITION_AWARE_REALTIME_ROUTING.equalsIgnoreCase(routingTableBuilderName))) { - SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig, propertyStore); + SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig); if (partitionSegmentPruner != null) { segmentPruners.add(partitionSegmentPruner); } @@ -102,8 +99,7 @@ public static List getSegmentPruners(TableConfig tableConfig, } @Nullable - private static SegmentPruner getPartitionSegmentPruner(TableConfig tableConfig, - ZkHelixPropertyStore propertyStore) { + private static SegmentPruner getPartitionSegmentPruner(TableConfig tableConfig) { String tableNameWithType = tableConfig.getTableName(); SegmentPartitionConfig segmentPartitionConfig = tableConfig.getIndexingConfig().getSegmentPartitionConfig(); if (segmentPartitionConfig == null) { @@ -137,26 +133,20 @@ private static TimeSegmentPruner getTimeSegmentPruner(TableConfig tableConfig, LOGGER.warn("Cannot enable time range pruning without time column for table: {}", tableNameWithType); return null; } - return createTimeSegmentPruner(tableConfig, propertyStore); - } - - @VisibleForTesting - static TimeSegmentPruner createTimeSegmentPruner(TableConfig tableConfig, - ZkHelixPropertyStore propertyStore) { - String tableNameWithType = tableConfig.getTableName(); - String timeColumn = tableConfig.getValidationConfig().getTimeColumnName(); - Preconditions.checkNotNull(timeColumn, "Time column must be configured in table config for table: %s", - tableNameWithType); - Schema schema = ZKMetadataProvider.getTableSchema(propertyStore, tableNameWithType); - Preconditions.checkNotNull(schema, "Failed to find schema for table: %s", tableNameWithType); - DateTimeFieldSpec dateTimeSpec = schema.getSpecForTimeColumn(timeColumn); - Preconditions.checkNotNull(dateTimeSpec, "Field spec must be specified in schema for time column: %s of table: %s", - timeColumn, tableNameWithType); - DateTimeFormatSpec timeFormatSpec = dateTimeSpec.getFormatSpec(); - - LOGGER.info("Using TimeRangePruner on time column: {} for table: {} with DateTimeFormatSpec: {}", - timeColumn, tableNameWithType, timeFormatSpec); - return new TimeSegmentPruner(tableConfig, timeColumn, timeFormatSpec); + Schema schema = ZKMetadataProvider.getTableSchema(propertyStore, tableConfig); + if (schema == null) { + LOGGER.warn("Cannot enable time range pruning without schema for table: {}", tableNameWithType); + return null; + } + DateTimeFieldSpec timeFieldSpec = schema.getSpecForTimeColumn(timeColumn); + if (timeFieldSpec == null) { + LOGGER.warn("Cannot enable time range pruning without field spec for table: {}, time column: {}", + tableNameWithType, timeColumn); + return null; + } + LOGGER.info("Using TimeRangePruner on time column: {} for table: {} with DateTimeFieldSpec: {}", timeColumn, + tableNameWithType, timeFieldSpec); + return new TimeSegmentPruner(tableConfig, timeFieldSpec); } private static List sortSegmentPruners(List pruners) { diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java index a7ac4fce4bdf..c2e6b20cce54 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.broker.routing.segmentpruner; +import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -37,7 +38,9 @@ import org.apache.pinot.common.request.Expression; import org.apache.pinot.common.request.Function; import org.apache.pinot.common.request.Identifier; +import org.apache.pinot.common.request.Literal; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.data.DateTimeFormatSpec; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.CommonConstants.Query.Range; @@ -64,10 +67,10 @@ public class TimeSegmentPruner implements SegmentPruner { private volatile IntervalTree _intervalTree; private final Map _intervalMap = new HashMap<>(); - public TimeSegmentPruner(TableConfig tableConfig, String timeColumn, DateTimeFormatSpec timeFormatSpec) { + public TimeSegmentPruner(TableConfig tableConfig, DateTimeFieldSpec timeFieldSpec) { _tableNameWithType = tableConfig.getTableName(); - _timeColumn = timeColumn; - _timeFormatSpec = timeFormatSpec; + _timeColumn = timeFieldSpec.getName(); + _timeFormatSpec = timeFieldSpec.getFormatSpec(); } @Override @@ -206,97 +209,53 @@ private List getFilterTimeIntervals(Expression filterExpression) { } else { return getComplementSortedIntervals(childIntervals); } - case EQUALS: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp, timeStamp)); - } else { - return null; + case EQUALS: + if (isTimeColumn(operands.get(0))) { + long timestamp = toMillisSinceEpoch(operands.get(1)); + return List.of(new Interval(timestamp, timestamp)); } - } - case IN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { + return null; + case IN: + if (isTimeColumn(operands.get(0))) { int numOperands = operands.size(); List intervals = new ArrayList<>(numOperands - 1); for (int i = 1; i < numOperands; i++) { - long timeStamp = - _timeFormatSpec.fromFormatToMillis(operands.get(i).getLiteral().getFieldValue().toString()); - intervals.add(new Interval(timeStamp, timeStamp)); + long timestamp = toMillisSinceEpoch(operands.get(i)); + intervals.add(new Interval(timestamp, timestamp)); } return intervals; - } else { - return null; } - } - case GREATER_THAN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp + 1, MAX_END_TIME)); - } else { - return null; + return null; + case GREATER_THAN: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)) + 1, MAX_END_TIME); } - } - case GREATER_THAN_OR_EQUAL: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp, MAX_END_TIME)); - } else { - return null; + return null; + case GREATER_THAN_OR_EQUAL: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)), MAX_END_TIME); } - } - case LESS_THAN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - if (timeStamp > MIN_START_TIME) { - return Collections.singletonList(new Interval(MIN_START_TIME, timeStamp - 1)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case LESS_THAN: + if (isTimeColumn(operands.get(0))) { + return getInterval(MIN_START_TIME, toMillisSinceEpoch(operands.get(1)) - 1); } - } - case LESS_THAN_OR_EQUAL: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - if (timeStamp >= MIN_START_TIME) { - return Collections.singletonList(new Interval(MIN_START_TIME, timeStamp)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case LESS_THAN_OR_EQUAL: + if (isTimeColumn(operands.get(0))) { + return getInterval(MIN_START_TIME, toMillisSinceEpoch(operands.get(1))); } - } - case BETWEEN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long startTimestamp = - _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - long endTimestamp = - _timeFormatSpec.fromFormatToMillis(operands.get(2).getLiteral().getFieldValue().toString()); - if (endTimestamp >= startTimestamp) { - return Collections.singletonList(new Interval(startTimestamp, endTimestamp)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case BETWEEN: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)), toMillisSinceEpoch(operands.get(2))); } - } - case RANGE: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { + return null; + case RANGE: + if (isTimeColumn(operands.get(0))) { return parseInterval(operands.get(1).getLiteral().getFieldValue().toString()); } return null; - } default: return null; } @@ -408,6 +367,17 @@ private List getComplementSortedIntervals(List intervals) { return res; } + private boolean isTimeColumn(Expression expression) { + Identifier identifier = expression.getIdentifier(); + return identifier != null && identifier.getName().equals(_timeColumn); + } + + private long toMillisSinceEpoch(Expression expression) { + Literal literal = expression.getLiteral(); + Preconditions.checkArgument(literal != null, "Literal is required for time column filter, got: %s", expression); + return _timeFormatSpec.fromFormatToMillis(literal.getFieldValue().toString()); + } + /** * Parse interval to millisecond as [min, max] with both sides included. * E.g. '(* 16311]' is parsed as [0, 16311], '(1455 16311)' is parsed as [1456, 16310] @@ -432,10 +402,10 @@ private List parseInterval(String rangeString) { endTime--; } } + return getInterval(startTime, endTime); + } - if (startTime > endTime) { - return Collections.emptyList(); - } - return Collections.singletonList(new Interval(startTime, endTime)); + private static List getInterval(long inclusiveStart, long inclusiveEnd) { + return inclusiveStart <= inclusiveEnd ? List.of(new Interval(inclusiveStart, inclusiveEnd)) : List.of(); } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java deleted file mode 100644 index b0fee613222a..000000000000 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java +++ /dev/null @@ -1,97 +0,0 @@ -/** - * 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.pinot.broker.broker; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeoutException; -import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; -import org.apache.pinot.spi.stream.PartitionGroupMetadata; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamMetadataProvider; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; - - -public class FakeStreamConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new FakePartitionLevelConsumer(); - } - @Override - public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { - return new FakesStreamMetadataProvider(); - } - - @Override - public StreamMetadataProvider createStreamMetadataProvider(String clientId) { - return new FakesStreamMetadataProvider(); - } - - public class FakePartitionLevelConsumer implements PartitionLevelConsumer { - - @Override - public MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) - throws TimeoutException { - return null; - } - - @Override - public void close() - throws IOException { - } - } - - public class FakesStreamMetadataProvider implements StreamMetadataProvider { - - @Override - public List computePartitionGroupMetadata(String clientId, StreamConfig streamConfig, - List partitionGroupConsumptionStatuses, int timeoutMillis) - throws IOException, TimeoutException { - return Collections.singletonList(new PartitionGroupMetadata(0, new LongMsgOffset(0))); - } - - @Override - public int fetchPartitionCount(long timeoutMillis) { - return 1; - } - - @Override - public Set fetchPartitionIds(long timeoutMillis) { - return Collections.singleton(0); - } - - @Override - public StreamPartitionMsgOffset fetchStreamPartitionOffset(OffsetCriteria offsetCriteria, long timeoutMillis) - throws TimeoutException { - return new LongMsgOffset(0); - } - - @Override - public void close() - throws IOException { - } - } -} diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java index 039d7a42053e..b08feaf87822 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java @@ -124,7 +124,7 @@ private Map getStreamConfigs() { streamConfigs.put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder"); streamConfigs.put("stream.kafka.consumer.factory.class.name", - "org.apache.pinot.broker.broker.FakeStreamConsumerFactory"); + "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); return streamConfigs; } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java index feaad35169ba..5e48a981ccc4 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java @@ -18,8 +18,7 @@ */ package org.apache.pinot.broker.routing.segmentpruner; -import java.util.Arrays; -import java.util.Collections; +import java.sql.Timestamp; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -50,11 +49,11 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.data.DateTimeFormatSpec; -import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; import org.mockito.Mockito; @@ -78,29 +77,45 @@ public class SegmentPrunerTest extends ControllerTest { private static final String SDF_PATTERN = "yyyyMMdd"; private static final String QUERY_1 = "SELECT * FROM testTable"; - private static final String QUERY_2 = "SELECT * FROM testTable where memberId = 0"; - private static final String QUERY_3 = "SELECT * FROM testTable where memberId IN (1, 2)"; - private static final String QUERY_4 = "SELECT * FROM testTable where memberId = 0 AND memberName='xyz'"; - - private static final String TIME_QUERY_1 = "SELECT * FROM testTable where timeColumn = 40"; - private static final String TIME_QUERY_2 = "SELECT * FROM testTable where timeColumn BETWEEN 20 AND 30"; - private static final String TIME_QUERY_3 = "SELECT * FROM testTable where 30 < timeColumn AND timeColumn <= 50"; - private static final String TIME_QUERY_4 = "SELECT * FROM testTable where timeColumn < 15 OR timeColumn > 45"; + private static final String QUERY_2 = "SELECT * FROM testTable WHERE memberId = 0"; + private static final String QUERY_3 = "SELECT * FROM testTable WHERE memberId IN (1, 2)"; + private static final String QUERY_4 = "SELECT * FROM testTable WHERE memberId = 0 AND memberName = 'xyz'"; + + private static final String TIME_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = 40"; + private static final String TIME_QUERY_2 = "SELECT * FROM testTable WHERE timeColumn BETWEEN 20 AND 30"; + private static final String TIME_QUERY_3 = "SELECT * FROM testTable WHERE 30 < timeColumn AND timeColumn <= 50"; + private static final String TIME_QUERY_4 = "SELECT * FROM testTable WHERE timeColumn < 15 OR timeColumn > 45"; private static final String TIME_QUERY_5 = - "SELECT * FROM testTable where timeColumn < 15 OR (60 < timeColumn AND timeColumn < 70)"; - private static final String TIME_QUERY_6 = "SELECT * FROM testTable where timeColumn < 0 AND timeColumn > 0"; + "SELECT * FROM testTable WHERE timeColumn < 15 OR (60 < timeColumn AND timeColumn < 70)"; + private static final String TIME_QUERY_6 = "SELECT * FROM testTable WHERE timeColumn NOT BETWEEN 20 AND 30"; + private static final String TIME_QUERY_7 = "SELECT * FROM testTable WHERE NOT timeColumn > 30"; + private static final String TIME_QUERY_8 = "SELECT * FROM testTable WHERE timeColumn < 0 AND timeColumn > 0"; - private static final String SDF_QUERY_1 = "SELECT * FROM testTable where timeColumn = 20200131"; - private static final String SDF_QUERY_2 = "SELECT * FROM testTable where timeColumn BETWEEN 20200101 AND 20200331"; + private static final String SDF_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = 20200131"; + private static final String SDF_QUERY_2 = "SELECT * FROM testTable WHERE timeColumn BETWEEN 20200101 AND 20200331"; private static final String SDF_QUERY_3 = - "SELECT * FROM testTable where 20200430 < timeColumn AND timeColumn < 20200630"; + "SELECT * FROM testTable WHERE 20200430 < timeColumn AND timeColumn < 20200630"; private static final String SDF_QUERY_4 = - "SELECT * FROM testTable where timeColumn <= 20200101 OR timeColumn in (20200201, 20200401)"; + "SELECT * FROM testTable WHERE timeColumn <= 20200101 OR timeColumn IN (20200201, 20200401)"; private static final String SDF_QUERY_5 = - "SELECT * FROM testTable where timeColumn in (20200101, 20200102) AND timeColumn >= 20200530"; - - private static final String SQL_TIME_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn NOT BETWEEN 20 AND 30"; - private static final String SQL_TIME_QUERY_2 = "SELECT * FROM testTable WHERE NOT timeColumn > 30"; + "SELECT * FROM testTable WHERE timeColumn IN (20200101, 20200102) AND timeColumn >= 20200530"; + + // Timestamp can be passed as string or long + private static final String TIMESTAMP_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = '2020-01-31 00:00:00'"; + private static final String TIMESTAMP_QUERY_2 = String.format("SELECT * FROM testTable WHERE timeColumn = %d", + Timestamp.valueOf("2020-01-31 00:00:00").getTime()); + private static final String TIMESTAMP_QUERY_3 = + "SELECT * FROM testTable WHERE timeColumn BETWEEN '2020-01-01 00:00:00' AND '2020-03-31 00:00:00'"; + private static final String TIMESTAMP_QUERY_4 = + String.format("SELECT * FROM testTable WHERE timeColumn BETWEEN %d AND %d", + Timestamp.valueOf("2020-01-01 00:00:00").getTime(), Timestamp.valueOf("2020-03-31 00:00:00").getTime()); + private static final String TIMESTAMP_QUERY_5 = + "SELECT * FROM testTable WHERE timeColumn <= '2020-01-01 00:00:00' OR timeColumn IN ('2020-02-01 00:00:00', " + + "'2020-04-01 00:00:00')"; + private static final String TIMESTAMP_QUERY_6 = + String.format("SELECT * FROM testTable WHERE timeColumn <= %d OR timeColumn IN (%d, %d)", + Timestamp.valueOf("2020-01-01 00:00:00").getTime(), Timestamp.valueOf("2020-02-01 00:00:00").getTime(), + Timestamp.valueOf("2020-04-01 00:00:00").getTime()); // this is duplicate with KinesisConfig.STREAM_TYPE, while instead of use KinesisConfig.STREAM_TYPE directly, we // hardcode the value here to avoid pulling the entire pinot-kinesis module as dependency. @@ -127,6 +142,7 @@ public void tearDown() { @Test public void testSegmentPrunerFactoryForPartitionPruner() { TableConfig tableConfig = mock(TableConfig.class); + when(tableConfig.getTableName()).thenReturn(OFFLINE_TABLE_NAME); IndexingConfig indexingConfig = mock(IndexingConfig.class); when(tableConfig.getIndexingConfig()).thenReturn(indexingConfig); @@ -141,8 +157,7 @@ public void testSegmentPrunerFactoryForPartitionPruner() { assertEquals(segmentPruners.size(), 0); // Segment partition config is missing - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE)); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE)); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); @@ -189,8 +204,7 @@ public void testSegmentPrunerFactoryForPartitionPruner() { @Test public void testSegmentPrunerFactoryForTimeRangePruner() { TableConfig tableConfig = mock(TableConfig.class); - when(tableConfig.getTableName()).thenReturn(RAW_TABLE_NAME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.HOURS); + when(tableConfig.getTableName()).thenReturn(OFFLINE_TABLE_NAME); // Routing config is missing List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); @@ -203,8 +217,7 @@ public void testSegmentPrunerFactoryForTimeRangePruner() { assertEquals(segmentPruners.size(), 0); // Validation config is missing - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.TIME_SEGMENT_PRUNER_TYPE)); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.TIME_SEGMENT_PRUNER_TYPE)); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); @@ -214,41 +227,54 @@ public void testSegmentPrunerFactoryForTimeRangePruner() { segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); - // Time range pruner should be returned + // Schema is missing when(validationConfig.getTimeColumnName()).thenReturn(TIME_COLUMN); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + assertEquals(segmentPruners.size(), 0); + + // Field spec is missing + Schema schema = new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME).build(); + ZKMetadataProvider.setSchema(_propertyStore, schema); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + assertEquals(segmentPruners.size(), 0); + + // Time range pruner should be returned + schema = new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME) + .addDateTimeField(TIME_COLUMN, DataType.TIMESTAMP, "TIMESTAMP", "1:MILLISECONDS").build(); + ZKMetadataProvider.setSchema(_propertyStore, schema); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof TimeSegmentPruner); } @Test - public void testEnablingEmptySegmentPruner() { + public void testSegmentPrunerFactoryForEmptySegmentPruner() { TableConfig tableConfig = mock(TableConfig.class); + when(tableConfig.getTableName()).thenReturn(REALTIME_TABLE_NAME); IndexingConfig indexingConfig = mock(IndexingConfig.class); + when(tableConfig.getIndexingConfig()).thenReturn(indexingConfig); RoutingConfig routingConfig = mock(RoutingConfig.class); - StreamIngestionConfig streamIngestionConfig = mock(StreamIngestionConfig.class); + when(tableConfig.getRoutingConfig()).thenReturn(routingConfig); + ZkHelixPropertyStore propertyStore = mock(ZkHelixPropertyStore.class); // When routingConfig is configured with EmptySegmentPruner, EmptySegmentPruner should be returned. - when(tableConfig.getRoutingConfig()).thenReturn(routingConfig); - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.EMPTY_SEGMENT_PRUNER_TYPE)); - List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.EMPTY_SEGMENT_PRUNER_TYPE)); + List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); // When indexingConfig is configured with Kinesis streaming, EmptySegmentPruner should be returned. - when(indexingConfig.getStreamConfigs()).thenReturn( - Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); - segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + when(indexingConfig.getStreamConfigs()).thenReturn(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); // When streamIngestionConfig is configured with Kinesis streaming, EmptySegmentPruner should be returned. + StreamIngestionConfig streamIngestionConfig = mock(StreamIngestionConfig.class); when(streamIngestionConfig.getStreamConfigMaps()).thenReturn( - Collections.singletonList(Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE))); - when(indexingConfig.getStreamConfigs()).thenReturn( - Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); - segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + List.of(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE))); + when(indexingConfig.getStreamConfigs()).thenReturn(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); } @@ -259,95 +285,76 @@ public void testPartitionAwareSegmentPruner() { BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_2); BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_3); BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_4); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); SinglePartitionColumnSegmentPruner singlePartitionColumnSegmentPruner = new SinglePartitionColumnSegmentPruner(OFFLINE_TABLE_NAME, PARTITION_COLUMN_1); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, - _propertyStore); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(singlePartitionColumnSegmentPruner); Set onlineSegments = new HashSet<>(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, Collections.emptySet()), - Collections.emptySet()); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, Collections.emptySet()), - Collections.emptySet()); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, Collections.emptySet()), - Collections.emptySet()); + + Set input = Set.of(); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Segments without metadata (not updated yet) should not be pruned String newSegment = "newSegment"; - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); + onlineSegments.add(newSegment); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + input = Set.of(newSegment); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Segments without partition metadata should not be pruned String segmentWithoutPartitionMetadata = "segmentWithoutPartitionMetadata"; - onlineSegments.add(segmentWithoutPartitionMetadata); - SegmentZKMetadata segmentZKMetadataWithoutPartitionMetadata = - new SegmentZKMetadata(segmentWithoutPartitionMetadata); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, OFFLINE_TABLE_NAME, - segmentZKMetadataWithoutPartitionMetadata); + new SegmentZKMetadata(segmentWithoutPartitionMetadata)); + onlineSegments.add(segmentWithoutPartitionMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); + input = Set.of(segmentWithoutPartitionMetadata); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Test different partition functions and number of partitions // 0 % 5 = 0; 1 % 5 = 1; 2 % 5 = 2 String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment0, "Modulo", 5, 0); + onlineSegments.add(segment0); // Murmur(0) % 4 = 0; Murmur(1) % 4 = 3; Murmur(2) % 4 = 0 String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment1, "Murmur", 4, 0); + onlineSegments.add(segment1); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); + input = Set.of(segment0, segment1); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), Set.of(segment1)); // Update partition metadata without refreshing should have no effect setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment0, "Modulo", 4, 1); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), Set.of(segment1)); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment0); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), Set.of(segment1)); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), Set.of(segment1)); // Multi-column partitioned segment. MultiPartitionColumnsSegmentPruner multiPartitionColumnsSegmentPruner = @@ -356,38 +363,25 @@ public void testPartitionAwareSegmentPruner() { segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(multiPartitionColumnsSegmentPruner); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, Collections.emptySet()), - Collections.emptySet()); + + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, input), Set.of(segment1)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, input), Set.of(segment1)); String segment2 = "segment2"; - onlineSegments.add(segment2); Map columnPartitionMetadataMap = new HashMap<>(); - columnPartitionMetadataMap.put(PARTITION_COLUMN_1, - new ColumnPartitionMetadata("Modulo", 4, Collections.singleton(0), null)); - Map partitionColumn2FunctionConfig = new HashMap<>(); - partitionColumn2FunctionConfig.put("columnValues", "xyz|abc"); - partitionColumn2FunctionConfig.put("columnValuesDelimiter", "|"); - columnPartitionMetadataMap.put(PARTITION_COLUMN_2, new ColumnPartitionMetadata( - "BoundedColumnValue", 3, Collections.singleton(1), partitionColumn2FunctionConfig)); + columnPartitionMetadataMap.put(PARTITION_COLUMN_1, new ColumnPartitionMetadata("Modulo", 4, Set.of(0), null)); + columnPartitionMetadataMap.put(PARTITION_COLUMN_2, new ColumnPartitionMetadata("BoundedColumnValue", 3, Set.of(1), + Map.of("columnValues", "xyz|abc", "columnValuesDelimiter", "|"))); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment2, columnPartitionMetadataMap); + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, - new HashSet<>(Arrays.asList(segment0, segment1, segment2))), new HashSet<>(Arrays.asList(segment1, segment2))); + input = Set.of(segment0, segment1, segment2); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, input), Set.of(segment1, segment2)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, input), Set.of(segment1, segment2)); } @Test @@ -399,143 +393,112 @@ public void testTimeSegmentPruner() { BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_4); BrokerRequest brokerRequest6 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_5); BrokerRequest brokerRequest7 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_6); + BrokerRequest brokerRequest8 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_7); + BrokerRequest brokerRequest9 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_8); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.DAYS); - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, - _propertyStore); - Set onlineSegments = new HashSet<>(); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + DateTimeFieldSpec timeFieldSpec = new DateTimeFieldSpec(TIME_COLUMN, DataType.INT, "EPOCH|DAYS", "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); + Set onlineSegments = new HashSet<>(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest4, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest5, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest6, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest7, Collections.emptySet()), Collections.emptySet()); - - // Initialize with non-empty onlineSegments + + Set input = Set.of(); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), input); + // Segments without metadata (not updated yet) should not be pruned - segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); - segmentZkMetadataFetcher.register(segmentPruner); String newSegment = "newSegment"; onlineSegments.add(newSegment); - segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest4, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest5, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest6, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest7, Collections.singleton(newSegment)), - Collections.emptySet()); // query with invalid range will always have empty filtered result + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + input = Set.of(newSegment); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Segments without time range metadata should not be pruned String segmentWithoutTimeRangeMetadata = "segmentWithoutTimeRangeMetadata"; - onlineSegments.add(segmentWithoutTimeRangeMetadata); SegmentZKMetadata segmentZKMetadataWithoutTimeRangeMetadata = new SegmentZKMetadata(segmentWithoutTimeRangeMetadata); - segmentZKMetadataWithoutTimeRangeMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, REALTIME_TABLE_NAME, segmentZKMetadataWithoutTimeRangeMetadata); + onlineSegments.add(segmentWithoutTimeRangeMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - segmentPruner.prune(brokerRequest1, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest2, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest3, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest4, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest5, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest6, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest7, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Test different time range String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, 10, 60, TimeUnit.DAYS); - + onlineSegments.add(segment0); String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, 20, 30, TimeUnit.DAYS); - + onlineSegments.add(segment1); String segment2 = "segment2"; - onlineSegments.add(segment2); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 50, 65, TimeUnit.DAYS); - + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest8, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Update metadata without external view change or refreshing should have no effect setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 20, 30, TimeUnit.DAYS); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest8, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment2); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range } @Test @@ -545,215 +508,175 @@ public void testTimeSegmentPrunerSimpleDateFormat() { BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_3); BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_4); BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_5); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpecSDF(RAW_TABLE_NAME, SDF_PATTERN); - - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + DateTimeFieldSpec timeFieldSpec = + new DateTimeFieldSpec(TIME_COLUMN, DataType.STRING, "SIMPLE_DATE_FORMAT|" + SDF_PATTERN, "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); - Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, RAW_TABLE_NAME); - DateTimeFormatSpec dateTimeFormatSpec = schema.getSpecForTimeColumn(TIME_COLUMN).getFormatSpec(); - + DateTimeFormatSpec timeFormatSpec = timeFieldSpec.getFormatSpec(); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, timeFormatSpec.fromFormatToMillis("20200101"), + timeFormatSpec.fromFormatToMillis("20200228"), TimeUnit.MILLISECONDS); onlineSegments.add(segment0); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, dateTimeFormatSpec.fromFormatToMillis("20200101"), - dateTimeFormatSpec.fromFormatToMillis("20200228"), TimeUnit.MILLISECONDS); - String segment1 = "segment1"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, timeFormatSpec.fromFormatToMillis("20200201"), + timeFormatSpec.fromFormatToMillis("20200530"), TimeUnit.MILLISECONDS); onlineSegments.add(segment1); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, dateTimeFormatSpec.fromFormatToMillis("20200201"), - dateTimeFormatSpec.fromFormatToMillis("20200530"), TimeUnit.MILLISECONDS); - String segment2 = "segment2"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, timeFormatSpec.fromFormatToMillis("20200401"), + timeFormatSpec.fromFormatToMillis("20200430"), TimeUnit.MILLISECONDS); onlineSegments.add(segment2); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, dateTimeFormatSpec.fromFormatToMillis("20200401"), - dateTimeFormatSpec.fromFormatToMillis("20200430"), TimeUnit.MILLISECONDS); - segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest2, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest3, onlineSegments), Collections.singleton(segment1)); - assertEquals(segmentPruner.prune(brokerRequest4, onlineSegments), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, onlineSegments), Collections.emptySet()); + + Set input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of()); } @Test - public void testTimeSegmentPrunerSql() { - BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(SQL_TIME_QUERY_1); - BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(SQL_TIME_QUERY_2); + public void testTimeSegmentPrunerTimestampFormat() { + BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_1); + BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_2); + BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_3); + BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_4); + BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_5); + BrokerRequest brokerRequest6 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_6); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.DAYS); - - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + // Intentionally put EPOCH as the format which Pinot should handle + DateTimeFieldSpec timeFieldSpec = + new DateTimeFieldSpec(TIME_COLUMN, DataType.TIMESTAMP, "EPOCH|MILLISECONDS", "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); + DateTimeFormatSpec timeFormatSpec = timeFieldSpec.getFormatSpec(); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, + timeFormatSpec.fromFormatToMillis("2020-01-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-02-28 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment0); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, 10, 60, TimeUnit.DAYS); String segment1 = "segment1"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, + timeFormatSpec.fromFormatToMillis("2020-02-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-05-30 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment1); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, 20, 30, TimeUnit.DAYS); String segment2 = "segment2"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, + timeFormatSpec.fromFormatToMillis("2020-04-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-04-30 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment2); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 50, 65, TimeUnit.DAYS); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment1))); + Set input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); } @Test public void testEmptySegmentPruner() { BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_1); - BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_2); - BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_3); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).build(); - // init with list of segments + // Init with a list of segments EmptySegmentPruner segmentPruner = new EmptySegmentPruner(tableConfig); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment0, 10); + onlineSegments.add(segment0); String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment1, 0); + onlineSegments.add(segment1); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - - // init with empty list of segments + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0)); + + // Init with no segment segmentPruner = new EmptySegmentPruner(tableConfig); segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); onlineSegments.clear(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.emptySet()), Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments without metadata (not updated yet) should not be pruned String newSegment = "newSegment"; onlineSegments.add(newSegment); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments without totalDocs metadata should not be pruned - onlineSegments.clear(); String segmentWithoutTotalDocsMetadata = "segmentWithoutTotalDocsMetadata"; - onlineSegments.add(segmentWithoutTotalDocsMetadata); SegmentZKMetadata segmentZKMetadataWithoutTotalDocsMetadata = new SegmentZKMetadata(segmentWithoutTotalDocsMetadata); - segmentZKMetadataWithoutTotalDocsMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, REALTIME_TABLE_NAME, segmentZKMetadataWithoutTotalDocsMetadata); + onlineSegments.add(segmentWithoutTotalDocsMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments with -1 totalDocs should not be pruned - onlineSegments.clear(); String segmentWithNegativeTotalDocsMetadata = "segmentWithNegativeTotalDocsMetadata"; - onlineSegments.add(segmentWithNegativeTotalDocsMetadata); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segmentWithNegativeTotalDocsMetadata, -1); + onlineSegments.add(segmentWithNegativeTotalDocsMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Prune segments with 0 total docs onlineSegments.clear(); - onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment0, 10); - onlineSegments.add(segment1); + onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment1, 0); + onlineSegments.add(segment1); String segment2 = "segment2"; - onlineSegments.add(segment2); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment2, -1); - + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0, segment2)); // Update metadata without external view change or refreshing should have no effect - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 20, 30, TimeUnit.DAYS); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment2, 0); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0, segment2)); // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment2); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - } - - private TableConfig getTableConfig(String rawTableName, TableType type) { - return new TableConfigBuilder(type).setTableName(rawTableName).setTimeColumnName(TIME_COLUMN).build(); - } - - private void setSchemaDateTimeFieldSpec(String rawTableName, TimeUnit timeUnit) { - ZKMetadataProvider.setSchema(_propertyStore, new Schema.SchemaBuilder().setSchemaName(rawTableName) - .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:" + timeUnit + ":EPOCH", "1:" + timeUnit).build()); - } - - private void setSchemaDateTimeFieldSpecSDF(String rawTableName, String format) { - ZKMetadataProvider.setSchema(_propertyStore, new Schema.SchemaBuilder().setSchemaName(rawTableName) - .addDateTime(TIME_COLUMN, FieldSpec.DataType.STRING, "1:DAYS:SIMPLE_DATE_FORMAT:" + format, "1:DAYS").build()); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0)); } private void setSegmentZKPartitionMetadata(String tableNameWithType, String segment, String partitionFunction, int numPartitions, int partitionId) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); - segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Collections.singletonMap(PARTITION_COLUMN_1, - new ColumnPartitionMetadata(partitionFunction, numPartitions, Collections.singleton(partitionId), null)))); + segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Map.of(PARTITION_COLUMN_1, + new ColumnPartitionMetadata(partitionFunction, numPartitions, Set.of(partitionId), null)))); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java index 45f34803a036..f0a1fdd1365c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java @@ -38,6 +38,7 @@ public enum ServerGauge implements AbstractMetrics.Gauge { LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS("seconds", false), LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS("seconds", false), REALTIME_OFFHEAP_MEMORY_USED("bytes", false), + REALTIME_MERGED_TEXT_IDX_DOCUMENT_AVG_LEN("bytes", false), REALTIME_SEGMENT_NUM_PARTITIONS("realtimeSegmentNumPartitions", false), LLC_SIMULTANEOUS_SEGMENT_BUILDS("llcSimultaneousSegmentBuilds", true), // Upsert metrics diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java index 02005a3814be..ed9769a68e61 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java @@ -40,6 +40,7 @@ public enum ServerMeter implements AbstractMetrics.Meter { INVALID_REALTIME_ROWS_DROPPED("rows", false), INCOMPLETE_REALTIME_ROWS_CONSUMED("rows", false), REALTIME_CONSUMPTION_EXCEPTIONS("exceptions", true), + REALTIME_MERGED_TEXT_IDX_TRUNCATED_DOCUMENT_SIZE("bytes", false), REALTIME_OFFSET_COMMITS("commits", true), REALTIME_OFFSET_COMMIT_EXCEPTIONS("exceptions", false), STREAM_CONSUMER_CREATE_EXCEPTIONS("exceptions", false), diff --git a/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java b/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java index 3260ced7b362..6e44f55ae168 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java @@ -52,6 +52,9 @@ public class MinionClient { private static final String ACCEPT = "accept"; private static final String APPLICATION_JSON = "application/json"; private static final String HTTP = "http"; + private static final TypeReference> TYPEREF_MAP_STRING_STRING = + new TypeReference>() { + }; private final String _controllerUrl; private final AuthProvider _authProvider; @@ -73,26 +76,25 @@ public Map scheduleMinionTasks(@Nullable String taskType, @Nulla int statusCode = response.getStatusLine().getStatusCode(); final String responseString = EntityUtils.toString(response.getEntity()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to schedule minion tasks. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to schedule minion tasks. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } public Map getTasksStates(String taskType) throws IOException, HttpException { - HttpGet httpGet = - createHttpGetRequest(MinionRequestURLBuilder.baseUrl(_controllerUrl).forTasksStates(taskType)); + HttpGet httpGet = createHttpGetRequest(MinionRequestURLBuilder.baseUrl(_controllerUrl).forTasksStates(taskType)); HttpResponse response = HTTP_CLIENT.execute(httpGet); int statusCode = response.getStatusLine().getStatusCode(); final String responseString = IOUtils.toString(response.getEntity().getContent()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } public String getTaskState(String taskName) @@ -102,8 +104,8 @@ public String getTaskState(String taskName) int statusCode = response.getStatusLine().getStatusCode(); String responseString = EntityUtils.toString(response.getEntity()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get state for task: %s. Error code %d, Error message: %s", taskName, statusCode, + throw new HttpException( + String.format("Unable to get state for task: %s. Error code %d, Error message: %s", taskName, statusCode, responseString)); } return responseString; @@ -121,11 +123,11 @@ public Map executeTask(AdhocTaskConfig adhocTaskConfig, @Nullabl int statusCode = response.getStatusLine().getStatusCode(); final String responseString = IOUtils.toString(response.getEntity().getContent(), Charset.defaultCharset()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } private HttpGet createHttpGetRequest(String uri) { diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java index 626a09100555..e8fd1287293c 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java @@ -221,8 +221,7 @@ public void testFetchFieldSpecForTime() { .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null) .addDateTime("dateTime0", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS") .addDateTime("dateTime1", FieldSpec.DataType.TIMESTAMP, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") - .addDateTime("dateTime2", FieldSpec.DataType.INT, "1:DAYS:EPOCH", "1:DAYS") - .build(); + .addDateTime("dateTime2", FieldSpec.DataType.INT, "1:DAYS:EPOCH", "1:DAYS").build(); // Test method which fetches the DateTimeFieldSpec given the timeColumnName // Test is on TIME @@ -254,7 +253,7 @@ public void testFetchFieldSpecForTime() { Assert.assertEquals(dateTimeFieldSpec.getDataType(), FieldSpec.DataType.TIMESTAMP); Assert.assertTrue(dateTimeFieldSpec.isSingleValueField()); Assert.assertEquals(dateTimeFieldSpec.getDefaultNullValue(), 0L); - Assert.assertEquals(dateTimeFieldSpec.getFormat(), "1:MILLISECONDS:EPOCH"); + Assert.assertEquals(dateTimeFieldSpec.getFormat(), "TIMESTAMP"); Assert.assertEquals(dateTimeFieldSpec.getGranularity(), "1:MILLISECONDS"); dateTimeFieldSpec = schema.getSpecForTimeColumn("dateTime2"); @@ -326,15 +325,10 @@ public void testSerializeDeserialize() @Test public void testSerializeDeserializeOptions() throws IOException { - String json = "{\n" - + " \"primaryKeyColumns\" : null,\n" - + " \"timeFieldSpec\" : null,\n" - + " \"schemaName\" : null,\n" - + " \"enableColumnBasedNullHandling\" : true,\n" - + " \"dimensionFieldSpecs\" : [ ],\n" - + " \"metricFieldSpecs\" : [ ],\n" - + " \"dateTimeFieldSpecs\" : [ ]\n" - + "}"; + String json = + "{\n" + " \"primaryKeyColumns\" : null,\n" + " \"timeFieldSpec\" : null,\n" + " \"schemaName\" : null,\n" + + " \"enableColumnBasedNullHandling\" : true,\n" + " \"dimensionFieldSpecs\" : [ ],\n" + + " \"metricFieldSpecs\" : [ ],\n" + " \"dateTimeFieldSpecs\" : [ ]\n" + "}"; JsonNode expectedNode = JsonUtils.stringToJsonNode(json); Schema schema = JsonUtils.jsonNodeToObject(expectedNode, Schema.class); @@ -363,6 +357,17 @@ public void testSimpleDateFormat() Assert.assertEquals(schemaFromJson.hashCode(), schema.hashCode()); } + @Test + public void testTimestampFormatOverride() + throws Exception { + URL resourceUrl = getClass().getClassLoader().getResource("schemaTest.schema"); + Assert.assertNotNull(resourceUrl); + Schema schema = Schema.fromFile(new File(resourceUrl.getFile())); + DateTimeFieldSpec fieldSpec = schema.getDateTimeSpec("dateTime3"); + Assert.assertNotNull(fieldSpec); + Assert.assertEquals(fieldSpec.getFormat(), "TIMESTAMP"); + } + @Test public void testByteType() throws Exception { diff --git a/pinot-core/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java b/pinot-common/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java similarity index 100% rename from pinot-core/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java rename to pinot-common/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java index 2ff578b5ee35..1b98a0851cff 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java @@ -30,7 +30,8 @@ import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig; import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamMessageDecoder; @@ -205,18 +206,20 @@ public GenericRow decode(byte[] payload, int offset, int length, GenericRow dest } private class TestStreamConsumerFactory extends StreamConsumerFactory { + @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { + public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return null; } @Override - public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { + public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return null; } @Override - public StreamMetadataProvider createStreamMetadataProvider(String clientId) { + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { return null; } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java index 7b32c46c9806..13f5b762b481 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java @@ -19,35 +19,50 @@ package org.apache.pinot.common.utils.request; import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.pinot.common.request.Expression; +import org.apache.pinot.common.request.ExpressionType; import org.apache.pinot.sql.parsers.PinotSqlType; import org.apache.pinot.sql.parsers.SqlNodeAndOptions; -import org.testng.Assert; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + public class RequestUtilsTest { + + @Test + public void testNullLiteralParsing() { + SqlLiteral nullLiteral = SqlLiteral.createNull(SqlParserPos.ZERO); + Expression nullExpr = RequestUtils.getLiteralExpression(nullLiteral); + assertEquals(nullExpr.getType(), ExpressionType.LITERAL); + assertTrue(nullExpr.getLiteral().getNullValue()); + } + // please check comments inside RequestUtils.getLiteralExpression() for why we need this test @Test public void testGetLiteralExpressionForObject() { Expression literalExpression = RequestUtils.getLiteralExpression(Float.valueOf(0.06f)); - Assert.assertEquals((literalExpression.getLiteral().getDoubleValue()), 0.06); + assertEquals((literalExpression.getLiteral().getDoubleValue()), 0.06); } @Test public void testGetLiteralExpressionForPrimitiveLong() { Expression literalExpression = RequestUtils.getLiteralExpression(4500L); - Assert.assertTrue(literalExpression.getLiteral().isSetLongValue()); - Assert.assertFalse(literalExpression.getLiteral().isSetDoubleValue()); - Assert.assertEquals(literalExpression.getLiteral().getLongValue(), 4500L); + assertTrue(literalExpression.getLiteral().isSetLongValue()); + assertFalse(literalExpression.getLiteral().isSetDoubleValue()); + assertEquals(literalExpression.getLiteral().getLongValue(), 4500L); } @Test public void testParseQuery() { SqlNodeAndOptions result = RequestUtils.parseQuery("select foo from countries where bar > 1"); - Assert.assertTrue(result.getParseTimeNs() > 0); - Assert.assertEquals(result.getSqlType(), PinotSqlType.DQL); - Assert.assertEquals(result.getSqlNode().toSqlString((SqlDialect) null).toString(), + assertTrue(result.getParseTimeNs() > 0); + assertEquals(result.getSqlType(), PinotSqlType.DQL); + assertEquals(result.getSqlNode().toSqlString((SqlDialect) null).toString(), "SELECT `foo`\n" + "FROM `countries`\n" + "WHERE `bar` > 1"); } } diff --git a/pinot-connectors/pinot-spark-2-connector/pom.xml b/pinot-connectors/pinot-spark-2-connector/pom.xml index a51582fcac05..14904aad2884 100644 --- a/pinot-connectors/pinot-spark-2-connector/pom.xml +++ b/pinot-connectors/pinot-spark-2-connector/pom.xml @@ -36,7 +36,7 @@ 2.4.8 2.8 1.3.0 - 3.1.1 + 3.2.18 org.apache.pinot.\$internal @@ -152,7 +152,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 diff --git a/pinot-connectors/pinot-spark-3-connector/pom.xml b/pinot-connectors/pinot-spark-3-connector/pom.xml index 6e53637cb270..a39548d22019 100644 --- a/pinot-connectors/pinot-spark-3-connector/pom.xml +++ b/pinot-connectors/pinot-spark-3-connector/pom.xml @@ -34,7 +34,7 @@ ${basedir}/../.. 3.5.1 - 3.1.1 + 3.2.18 org.apache.pinot.\$internal @@ -148,7 +148,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 diff --git a/pinot-connectors/pinot-spark-common/pom.xml b/pinot-connectors/pinot-spark-common/pom.xml index bf60a2f6adfa..a4f72ed076e4 100644 --- a/pinot-connectors/pinot-spark-common/pom.xml +++ b/pinot-connectors/pinot-spark-common/pom.xml @@ -36,7 +36,7 @@ 0.14.6 2.8 1.3.0 - 3.1.1 + 3.2.18 @@ -163,7 +163,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 diff --git a/pinot-controller/pom.xml b/pinot-controller/pom.xml index 9b995c23e971..f70dde647129 100644 --- a/pinot-controller/pom.xml +++ b/pinot-controller/pom.xml @@ -144,7 +144,7 @@ com.github.eirslett frontend-maven-plugin - 1.11.0 + 1.15.0 diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java index f78a49f2f927..91df2e07431d 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java @@ -201,8 +201,8 @@ public void init(PinotConfiguration pinotConfiguration) _helixClusterName = _config.getHelixClusterName(); ServiceStartableUtils.applyClusterConfig(_config, _helixZkURL, _helixClusterName, ServiceRole.CONTROLLER); - PinotInsecureMode.setPinotInInsecureMode( - Boolean.valueOf(_config.getProperty(CommonConstants.CONFIG_OF_PINOT_INSECURE_MODE, + PinotInsecureMode.setPinotInInsecureMode(Boolean.valueOf( + _config.getProperty(CommonConstants.CONFIG_OF_PINOT_INSECURE_MODE, CommonConstants.DEFAULT_PINOT_INSECURE_MODE))); setupHelixSystemProperties(); @@ -531,8 +531,8 @@ protected void configure() { if (tableConfig != null) { Map streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig); try { - StreamConfig.validateConsumerType( - streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"), streamConfigMap); + StreamConfig.validateConsumerType(streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"), + streamConfigMap); } catch (Exception e) { existingHlcTables.add(rt); } @@ -587,66 +587,63 @@ public void fixSchemaNameInTableConfig() { AtomicInteger failedToUpdateTableConfigCount = new AtomicInteger(); ZkHelixPropertyStore propertyStore = _helixResourceManager.getPropertyStore(); - _helixResourceManager.getDatabaseNames().stream() - .map(_helixResourceManager::getAllTables) - .flatMap(List::stream) - .forEach(tableNameWithType -> { - Pair tableConfigWithVersion = - ZKMetadataProvider.getTableConfigWithVersion(propertyStore, tableNameWithType); - if (tableConfigWithVersion == null) { - // This might due to table deletion, just log it here. - LOGGER.warn("Failed to find table config for table: {}, the table likely already got deleted", - tableNameWithType); - return; - } - TableConfig tableConfig = tableConfigWithVersion.getLeft(); - String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); - String schemaPath = ZKMetadataProvider.constructPropertyStorePathForSchema(rawTableName); - boolean schemaExists = propertyStore.exists(schemaPath, AccessOption.PERSISTENT); - String existSchemaName = tableConfig.getValidationConfig().getSchemaName(); - if (existSchemaName == null || existSchemaName.equals(rawTableName)) { - // Although the table config is valid, we still need to ensure the schema exists - if (!schemaExists) { - LOGGER.warn("Failed to find schema for table: {}", tableNameWithType); - tableWithoutSchemaCount.getAndIncrement(); - return; - } - // Table config is already in good status - return; - } - misconfiguredTableCount.getAndIncrement(); - if (schemaExists) { - // If a schema named `rawTableName` already exists, then likely this is a misconfiguration. - // Reset schema name in table config to null to let the table point to the existing schema. - LOGGER.warn("Schema: {} already exists, fix the schema name in table config from {} to null", rawTableName, - existSchemaName); - } else { - // Copy the schema current table referring to to `rawTableName` if it does not exist - Schema schema = _helixResourceManager.getSchema(existSchemaName); - if (schema == null) { - LOGGER.warn("Failed to find schema: {} for table: {}", existSchemaName, tableNameWithType); - tableWithoutSchemaCount.getAndIncrement(); - return; - } - schema.setSchemaName(rawTableName); - if (propertyStore.create(schemaPath, SchemaUtils.toZNRecord(schema), AccessOption.PERSISTENT)) { - LOGGER.info("Copied schema: {} to {}", existSchemaName, rawTableName); - } else { - LOGGER.warn("Failed to copy schema: {} to {}", existSchemaName, rawTableName); - failedToCopySchemaCount.getAndIncrement(); - return; - } - } - // Update table config to remove schema name - tableConfig.getValidationConfig().setSchemaName(null); - if (ZKMetadataProvider.setTableConfig(propertyStore, tableConfig, tableConfigWithVersion.getRight())) { - LOGGER.info("Removed schema name from table config for table: {}", tableNameWithType); - fixedSchemaTableCount.getAndIncrement(); - } else { - LOGGER.warn("Failed to update table config for table: {}", tableNameWithType); - failedToUpdateTableConfigCount.getAndIncrement(); - } - }); + _helixResourceManager.getAllTables().forEach(tableNameWithType -> { + Pair tableConfigWithVersion = + ZKMetadataProvider.getTableConfigWithVersion(propertyStore, tableNameWithType); + if (tableConfigWithVersion == null) { + // This might due to table deletion, just log it here. + LOGGER.warn("Failed to find table config for table: {}, the table likely already got deleted", + tableNameWithType); + return; + } + TableConfig tableConfig = tableConfigWithVersion.getLeft(); + String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); + String schemaPath = ZKMetadataProvider.constructPropertyStorePathForSchema(rawTableName); + boolean schemaExists = propertyStore.exists(schemaPath, AccessOption.PERSISTENT); + String existSchemaName = tableConfig.getValidationConfig().getSchemaName(); + if (existSchemaName == null || existSchemaName.equals(rawTableName)) { + // Although the table config is valid, we still need to ensure the schema exists + if (!schemaExists) { + LOGGER.warn("Failed to find schema for table: {}", tableNameWithType); + tableWithoutSchemaCount.getAndIncrement(); + return; + } + // Table config is already in good status + return; + } + misconfiguredTableCount.getAndIncrement(); + if (schemaExists) { + // If a schema named `rawTableName` already exists, then likely this is a misconfiguration. + // Reset schema name in table config to null to let the table point to the existing schema. + LOGGER.warn("Schema: {} already exists, fix the schema name in table config from {} to null", rawTableName, + existSchemaName); + } else { + // Copy the schema current table referring to to `rawTableName` if it does not exist + Schema schema = _helixResourceManager.getSchema(existSchemaName); + if (schema == null) { + LOGGER.warn("Failed to find schema: {} for table: {}", existSchemaName, tableNameWithType); + tableWithoutSchemaCount.getAndIncrement(); + return; + } + schema.setSchemaName(rawTableName); + if (propertyStore.create(schemaPath, SchemaUtils.toZNRecord(schema), AccessOption.PERSISTENT)) { + LOGGER.info("Copied schema: {} to {}", existSchemaName, rawTableName); + } else { + LOGGER.warn("Failed to copy schema: {} to {}", existSchemaName, rawTableName); + failedToCopySchemaCount.getAndIncrement(); + return; + } + } + // Update table config to remove schema name + tableConfig.getValidationConfig().setSchemaName(null); + if (ZKMetadataProvider.setTableConfig(propertyStore, tableConfig, tableConfigWithVersion.getRight())) { + LOGGER.info("Removed schema name from table config for table: {}", tableNameWithType); + fixedSchemaTableCount.getAndIncrement(); + } else { + LOGGER.warn("Failed to update table config for table: {}", tableNameWithType); + failedToUpdateTableConfigCount.getAndIncrement(); + } + }); LOGGER.info( "Found {} tables misconfigured, {} tables without schema. Successfully fixed schema for {} tables, failed to " + "fix {} tables due to copy schema failure, failed to fix {} tables due to update table config failure.", diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java index 1863819ab97e..4fdca71a8200 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java @@ -127,10 +127,8 @@ public List getBrokersForTenant( @ApiOperation(value = "List tables to brokers mappings", notes = "List tables to brokers mappings") public Map> getTablesToBrokersMapping( @ApiParam(value = "ONLINE|OFFLINE") @QueryParam("state") String state, @Context HttpHeaders headers) { - Map> resultMap = new HashMap<>(); - _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)) - .forEach(table -> resultMap.put(table, getBrokersForTable(table, null, state, headers))); - return resultMap; + return _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)).stream() + .collect(Collectors.toMap(table -> table, table -> getBrokersForTable(table, null, state, headers))); } @GET @@ -201,11 +199,8 @@ public List getBrokersForTenantV2( @ApiOperation(value = "List tables to brokers mappings", notes = "List tables to brokers mappings") public Map> getTablesToBrokersMappingV2( @ApiParam(value = "ONLINE|OFFLINE") @QueryParam("state") String state, @Context HttpHeaders headers) { - Map> resultMap = new HashMap<>(); - String databaseName = headers.getHeaderString(DATABASE); - _pinotHelixResourceManager.getAllRawTables(databaseName).stream() - .forEach(table -> resultMap.put(table, getBrokersForTableV2(table, null, state, headers))); - return resultMap; + return _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)).stream() + .collect(Collectors.toMap(table -> table, table -> getBrokersForTableV2(table, null, state, headers))); } @GET diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java index 5f09f74336a9..16f13e047165 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java @@ -109,8 +109,7 @@ public LeadControllerResponse getLeadersForAllTables(@Context HttpHeaders header } // Assigns all the tables to the relevant partitions. - List tableNames = _pinotHelixResourceManager.getAllTables( - headers.getHeaderString(DATABASE)); + List tableNames = _pinotHelixResourceManager.getAllTables(headers.getHeaderString(DATABASE)); for (String tableName : tableNames) { String rawTableName = TableNameBuilder.extractRawTableName(tableName); int partitionId = LeadControllerUtils.getPartitionIdForTable(rawTableName); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java index dbd1df943827..aa40ec40a8d4 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java @@ -49,11 +49,11 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.calcite.sql.SqlNode; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.helix.model.InstanceConfig; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.Utils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.response.ProcessingException; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java index 0d41216aa5f2..0b8725b10e71 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java @@ -588,7 +588,7 @@ public String changeTenantState( } } - boolean enable = StateType.ENABLE.name().equalsIgnoreCase(state) ? true : false; + boolean enable = StateType.ENABLE.name().equalsIgnoreCase(state); ObjectNode instanceResult = JsonUtils.newObjectNode(); String instance = null; try { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index f1761ce8666e..680e475d6353 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -716,13 +716,12 @@ public List getAllResources() { } /** - * Get all table names (with type suffix) in default database. + * Get all table names (with type suffix) in all databases. * - * @return List of table names in default database + * @return List of table names */ - @Deprecated public List getAllTables() { - return getAllTables(null); + return getAllResources().stream().filter(TableNameBuilder::isTableResource).collect(Collectors.toList()); } /** @@ -732,23 +731,18 @@ public List getAllTables() { * @return List of table names in provided database name */ public List getAllTables(@Nullable String databaseName) { - List tableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) - && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { - tableNames.add(resourceName); - } - } - return tableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isTableResource(resourceName) && DatabaseUtils.isPartOfDatabase(resourceName, + databaseName)).collect(Collectors.toList()); } /** - * Get all offline table names from default database. + * Get all offline table names from all databases. * - * @return List of offline table names in default database + * @return List of offline table names */ public List getAllOfflineTables() { - return getAllOfflineTables(null); + return getAllResources().stream().filter(TableNameBuilder::isOfflineTableResource).collect(Collectors.toList()); } /** @@ -758,23 +752,18 @@ public List getAllOfflineTables() { * @return List of offline table names in provided database name */ public List getAllOfflineTables(@Nullable String databaseName) { - List offlineTableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) - && TableNameBuilder.isOfflineTableResource(resourceName)) { - offlineTableNames.add(resourceName); - } - } - return offlineTableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isOfflineTableResource(resourceName) && DatabaseUtils.isPartOfDatabase( + resourceName, databaseName)).collect(Collectors.toList()); } /** - * Get all dimension table names from default database. + * Get all dimension table names from all databases. * - * @return List of dimension table names in default database + * @return List of dimension table names */ public List getAllDimensionTables() { - return getAllDimensionTables(null); + return _tableCache.getAllDimensionTables(); } /** @@ -785,17 +774,16 @@ public List getAllDimensionTables() { */ public List getAllDimensionTables(@Nullable String databaseName) { return _tableCache.getAllDimensionTables().stream() - .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)) - .collect(Collectors.toList()); + .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)).collect(Collectors.toList()); } /** - * Get all realtime table names from default database. + * Get all realtime table names from all databases. * - * @return List of realtime table names in default database + * @return List of realtime table names */ public List getAllRealtimeTables() { - return getAllRealtimeTables(null); + return getAllResources().stream().filter(TableNameBuilder::isRealtimeTableResource).collect(Collectors.toList()); } /** @@ -805,23 +793,19 @@ public List getAllRealtimeTables() { * @return List of realtime table names in provided database name */ public List getAllRealtimeTables(@Nullable String databaseName) { - List realtimeTableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) - && TableNameBuilder.isRealtimeTableResource(resourceName)) { - realtimeTableNames.add(resourceName); - } - } - return realtimeTableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isRealtimeTableResource(resourceName) && DatabaseUtils.isPartOfDatabase( + resourceName, databaseName)).collect(Collectors.toList()); } /** - * Get all raw table names in default database. + * Get all raw table names in all databases. * - * @return List of raw table names in default database + * @return List of raw table names */ public List getAllRawTables() { - return getAllRawTables(null); + return getAllResources().stream().filter(TableNameBuilder::isTableResource) + .map(TableNameBuilder::extractRawTableName).distinct().collect(Collectors.toList()); } /** @@ -831,14 +815,9 @@ public List getAllRawTables() { * @return List of raw table names in provided database name */ public List getAllRawTables(@Nullable String databaseName) { - Set rawTableNames = new HashSet<>(); - for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) - && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { - rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); - } - } - return new ArrayList<>(rawTableNames); + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isTableResource(resourceName) && DatabaseUtils.isPartOfDatabase(resourceName, + databaseName)).map(TableNameBuilder::extractRawTableName).distinct().collect(Collectors.toList()); } /** diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java index 027712bde88b..b25746298549 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java @@ -138,12 +138,9 @@ private Set getBrokerInstancesInUse() { private Set getServerInstancesInUse() { Set serverInstancesInUse = new HashSet<>(); - _pinotHelixResourceManager.getDatabaseNames().stream() - .map(_pinotHelixResourceManager::getAllTables) - .flatMap(List::stream) - .forEach(tableName -> serverInstancesInUse.addAll( - Optional.ofNullable(_pinotHelixResourceManager.getTableIdealState(tableName)) - .map(is -> is.getInstanceSet(tableName)).orElse(Collections.emptySet()))); + _pinotHelixResourceManager.getAllTables().forEach(tableName -> serverInstancesInUse.addAll( + Optional.ofNullable(_pinotHelixResourceManager.getTableIdealState(tableName)) + .map(is -> is.getInstanceSet(tableName)).orElse(Collections.emptySet()))); return serverInstancesInUse; } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java index 9b2fced8c28a..2cdbf8c1dfe2 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java @@ -480,13 +480,12 @@ public void registerTaskGenerator(PinotTaskGenerator taskGenerator) { } /** - * Public API to schedule tasks (all task types) for all tables in default database. + * Public API to schedule tasks (all task types) for all tables in all databases. * It might be called from the non-leader controller. * Returns a map from the task type to the list of tasks scheduled. */ - @Deprecated public synchronized Map> scheduleTasks() { - return scheduleTasks(_pinotHelixResourceManager.getAllTables(CommonConstants.DEFAULT_DATABASE), false); + return scheduleTasks(_pinotHelixResourceManager.getAllTables(), false); } /** @@ -494,7 +493,7 @@ public synchronized Map> scheduleTasks() { * It might be called from the non-leader controller. * Returns a map from the task type to the list of tasks scheduled. */ - public synchronized Map> scheduleTasksForDatabase(String database) { + public synchronized Map> scheduleTasksForDatabase(@Nullable String database) { return scheduleTasks(_pinotHelixResourceManager.getAllTables(database), false); } @@ -605,8 +604,8 @@ private List scheduleTask(PinotTaskGenerator taskGenerator, List 0) { @@ -629,14 +628,13 @@ public synchronized Map> scheduleTasks(String tableNameWith } /** - * Public API to schedule task for the given task type in default database. + * Public API to schedule task for the given task type in all databases. * It might be called from the non-leader controller. * Returns the list of task names, or {@code null} if no task is scheduled. */ - @Deprecated @Nullable public synchronized List scheduleTask(String taskType) { - return scheduleTaskForDatabase(taskType, CommonConstants.DEFAULT_DATABASE); + return scheduleTask(taskType, _pinotHelixResourceManager.getAllTables()); } /** @@ -645,13 +643,18 @@ public synchronized List scheduleTask(String taskType) { * Returns the list of task name, or {@code null} if no task is scheduled. */ @Nullable - public synchronized List scheduleTaskForDatabase(String taskType, String database) { + public synchronized List scheduleTaskForDatabase(String taskType, @Nullable String database) { + return scheduleTask(taskType, _pinotHelixResourceManager.getAllTables(database)); + } + + @Nullable + private List scheduleTask(String taskType, List tables) { PinotTaskGenerator taskGenerator = _taskGeneratorRegistry.getTaskGenerator(taskType); Preconditions.checkState(taskGenerator != null, "Task type: %s is not registered", taskType); // Scan all table configs to get the tables with task enabled List enabledTableConfigs = new ArrayList<>(); - for (String tableNameWithType : _pinotHelixResourceManager.getAllTables(database)) { + for (String tableNameWithType : tables) { TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); if (tableConfig != null && tableConfig.getTaskConfig() != null && tableConfig.getTaskConfig() .isTaskTypeEnabled(taskType)) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java index 47dc218f43e5..d3584df0682e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java @@ -20,7 +20,6 @@ import com.google.common.collect.Sets; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Properties; @@ -69,12 +68,8 @@ protected final void runTask(Properties periodicTaskProperties) { // Check if we have a specific table against which this task needs to be run. String propTableNameWithType = (String) periodicTaskProperties.get(PeriodicTask.PROPERTY_KEY_TABLE_NAME); // Process the tables that are managed by this controller - List allTables = propTableNameWithType == null - ? _pinotHelixResourceManager.getDatabaseNames().stream() - .map(_pinotHelixResourceManager::getAllTables) - .flatMap(List::stream) - .collect(Collectors.toList()) - : Collections.singletonList(propTableNameWithType); + List allTables = + propTableNameWithType != null ? List.of(propTableNameWithType) : _pinotHelixResourceManager.getAllTables(); Set currentLeaderOfTables = allTables.stream() .filter(_leadControllerManager::isLeaderForTable) diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java index 051fd784b685..26928deb3c0f 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java @@ -18,9 +18,6 @@ */ package org.apache.pinot.controller.helix; -import com.google.common.collect.ImmutableMap; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -43,15 +40,14 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; public class RealtimeConsumerMonitorTest { @@ -59,17 +55,16 @@ public class RealtimeConsumerMonitorTest { @Test public void realtimeBasicTest() throws Exception { - final String tableName = "myTable_REALTIME"; - final String rawTableName = TableNameBuilder.extractRawTableName(tableName); - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String rawTableName = "myTable"; + String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setTimeColumnName("timeColumn") + new TableConfigBuilder(TableType.REALTIME).setTableName(rawTableName).setTimeColumnName("timeColumn") .setNumReplicas(2).setStreamConfigs(getStreamConfigMap()).build(); + LLCSegmentName segmentPartition1Seq0 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); LLCSegmentName segmentPartition1Seq1 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); LLCSegmentName segmentPartition2Seq0 = new LLCSegmentName(rawTableName, 2, 0, System.currentTimeMillis()); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState(segmentPartition1Seq0.getSegmentName(), "pinot1", "ONLINE"); idealState.setPartitionState(segmentPartition1Seq0.getSegmentName(), "pinot2", "ONLINE"); idealState.setPartitionState(segmentPartition1Seq1.getSegmentName(), "pinot1", "CONSUMING"); @@ -79,7 +74,7 @@ public void realtimeBasicTest() idealState.setReplicas("3"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(realtimeTableName); externalView.setState(segmentPartition1Seq0.getSegmentName(), "pinot1", "ONLINE"); externalView.setState(segmentPartition1Seq0.getSegmentName(), "pinot2", "ONLINE"); externalView.setState(segmentPartition1Seq1.getSegmentName(), "pinot1", "CONSUMING"); @@ -91,13 +86,11 @@ public void realtimeBasicTest() { helixResourceManager = mock(PinotHelixResourceManager.class); ZkHelixPropertyStore helixPropertyStore = mock(ZkHelixPropertyStore.class); - when(helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); + when(helixResourceManager.getTableConfig(realtimeTableName)).thenReturn(tableConfig); when(helixResourceManager.getPropertyStore()).thenReturn(helixPropertyStore); - when(helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(externalView); ZNRecord znRecord = new ZNRecord("0"); znRecord.setSimpleField(CommonConstants.Segment.Realtime.END_OFFSET, "10000"); when(helixPropertyStore.get(anyString(), any(), anyInt())).thenReturn(znRecord); @@ -121,61 +114,53 @@ public void realtimeBasicTest() // So, the consumer monitor should show: 1. partition-1 has 0 lag; partition-2 has some non-zero lag. // Segment 1 in replicas: TreeMap> response = new TreeMap<>(); - List part1ServerConsumingSegmentInfo = new ArrayList<>(2); - part1ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot1", "1", "100", "100", "0")); - part1ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot2", "1", "100", "100", "0")); - + List part1ServerConsumingSegmentInfo = + List.of(getConsumingSegmentInfoForServer("pinot1", "1", "100", "100", "0"), + getConsumingSegmentInfoForServer("pinot2", "1", "100", "100", "0")); response.put(segmentPartition1Seq1.getSegmentName(), part1ServerConsumingSegmentInfo); // Segment 2 in replicas - List part2ServerConsumingSegmentInfo = new ArrayList<>(2); - part2ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot1", "2", "120", "120", "0")); - part2ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot2", "2", "80", "120", "60000")); - + List part2ServerConsumingSegmentInfo = + List.of(getConsumingSegmentInfoForServer("pinot1", "2", "120", "120", "0"), + getConsumingSegmentInfoForServer("pinot2", "2", "80", "120", "60000")); response.put(segmentPartition2Seq0.getSegmentName(), part2ServerConsumingSegmentInfo); ConsumingSegmentInfoReader consumingSegmentReader = mock(ConsumingSegmentInfoReader.class); - when(consumingSegmentReader.getConsumingSegmentsInfo(tableName, 10000)) - .thenReturn(new ConsumingSegmentInfoReader.ConsumingSegmentsInfoMap(response, 0, 0)); + when(consumingSegmentReader.getConsumingSegmentsInfo(realtimeTableName, 10000)).thenReturn( + new ConsumingSegmentInfoReader.ConsumingSegmentsInfoMap(response, 0, 0)); RealtimeConsumerMonitor realtimeConsumerMonitor = - new RealtimeConsumerMonitor(config, helixResourceManager, leadControllerManager, - controllerMetrics, consumingSegmentReader); + new RealtimeConsumerMonitor(config, helixResourceManager, leadControllerManager, controllerMetrics, + consumingSegmentReader); realtimeConsumerMonitor.start(); realtimeConsumerMonitor.run(); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 1, + + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 1, ControllerGauge.MAX_RECORDS_LAG), 0); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 2, + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 2, ControllerGauge.MAX_RECORDS_LAG), 40); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 1, - ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 0); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 2, + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 1, + ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 0); + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 2, ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 60000); } ConsumingSegmentInfoReader.ConsumingSegmentInfo getConsumingSegmentInfoForServer(String serverName, String partitionId, String currentOffset, String upstreamLatestOffset, String availabilityLagMs) { - Map currentOffsetMap = Collections.singletonMap(partitionId, currentOffset); - Map latestUpstreamOffsetMap = Collections.singletonMap(partitionId, upstreamLatestOffset); - Map recordsLagMap = Collections.singletonMap(partitionId, String.valueOf( - Long.parseLong(upstreamLatestOffset) - Long.parseLong(currentOffset))); - Map availabilityLagMsMap = Collections.singletonMap(partitionId, availabilityLagMs); + Map currentOffsetMap = Map.of(partitionId, currentOffset); + Map latestUpstreamOffsetMap = Map.of(partitionId, upstreamLatestOffset); + Map recordsLagMap = + Map.of(partitionId, String.valueOf(Long.parseLong(upstreamLatestOffset) - Long.parseLong(currentOffset))); + Map availabilityLagMsMap = Map.of(partitionId, availabilityLagMs); ConsumingSegmentInfoReader.PartitionOffsetInfo partitionOffsetInfo = new ConsumingSegmentInfoReader.PartitionOffsetInfo(currentOffsetMap, latestUpstreamOffsetMap, recordsLagMap, availabilityLagMsMap); - return new ConsumingSegmentInfoReader.ConsumingSegmentInfo(serverName, "CONSUMING", -1, - currentOffsetMap, partitionOffsetInfo); + return new ConsumingSegmentInfoReader.ConsumingSegmentInfo(serverName, "CONSUMING", -1, currentOffsetMap, + partitionOffsetInfo); } Map getStreamConfigMap() { - return ImmutableMap.of( - "streamType", "kafka", - "stream.kafka.consumer.type", "simple", - "stream.kafka.topic.name", "test", + return Map.of("streamType", "kafka", "stream.kafka.consumer.type", "simple", "stream.kafka.topic.name", "test", "stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", "stream.kafka.consumer.factory.class.name", "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java index a1dd8f2697b0..3161c9da200a 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java @@ -18,10 +18,6 @@ */ package org.apache.pinot.controller.helix; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -52,19 +48,21 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; public class SegmentStatusCheckerTest { + private final ExecutorService _executorService = Executors.newFixedThreadPool(1); + private SegmentStatusChecker _segmentStatusChecker; private PinotHelixResourceManager _helixResourceManager; private ZkHelixPropertyStore _helixPropertyStore; @@ -73,18 +71,15 @@ public class SegmentStatusCheckerTest { private ControllerMetrics _controllerMetrics; private ControllerConf _config; private TableSizeReader _tableSizeReader; - private ExecutorService _executorService = Executors.newFixedThreadPool(1); @Test public void offlineBasicTest() throws Exception { - final String tableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String offlineTableName = "myTable_OFFLINE"; TableConfig tableConfig = - new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setNumReplicas(2).build(); + new TableConfigBuilder(TableType.OFFLINE).setTableName(offlineTableName).setNumReplicas(2).build(); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot3", "ONLINE"); @@ -101,7 +96,7 @@ public void offlineBasicTest() idealState.setReplicas("2"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(offlineTableName); externalView.setState("myTable_0", "pinot1", "ONLINE"); externalView.setState("myTable_0", "pinot2", "ONLINE"); externalView.setState("myTable_1", "pinot1", "ERROR"); @@ -114,27 +109,23 @@ public void offlineBasicTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableConfig(offlineTableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); } { _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); // Based on the lineage entries: {myTable_1 -> myTable_3, COMPLETED}, {myTable_3 -> myTable_4, IN_PROGRESS}, // myTable_1 and myTable_4 will be skipped for the metrics. - SegmentLineage segmentLineage = new SegmentLineage(tableName); + SegmentLineage segmentLineage = new SegmentLineage(offlineTableName); segmentLineage.addLineageEntry(SegmentLineageUtils.generateLineageEntryId(), - new LineageEntry(Collections.singletonList("myTable_1"), Collections.singletonList("myTable_3"), - LineageEntryState.COMPLETED, 11111L)); + new LineageEntry(List.of("myTable_1"), List.of("myTable_3"), LineageEntryState.COMPLETED, 11111L)); segmentLineage.addLineageEntry(SegmentLineageUtils.generateLineageEntryId(), - new LineageEntry(Collections.singletonList("myTable_3"), Collections.singletonList("myTable_4"), - LineageEntryState.IN_PROGRESS, 11111L)); - when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + tableName), any(), eq(AccessOption.PERSISTENT))) - .thenReturn(segmentLineage.toZNRecord()); + new LineageEntry(List.of("myTable_3"), List.of("myTable_4"), LineageEntryState.IN_PROGRESS, 11111L)); + when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + offlineTableName), any(), + eq(AccessOption.PERSISTENT))).thenReturn(segmentLineage.toZNRecord()); } { _config = mock(ControllerConf.class); @@ -158,40 +149,41 @@ public void offlineBasicTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, offlineTableName, ControllerGauge.REPLICATION_FROM_CONFIG), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENT_COUNT), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED), 5); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENT_COUNT), + 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED), 5); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 66); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 66); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void realtimeBasicTest() throws Exception { - final String tableName = "myTable_REALTIME"; - final String rawTableName = TableNameBuilder.extractRawTableName(tableName); - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String rawTableName = "myTable"; + String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setTimeColumnName("timeColumn") + new TableConfigBuilder(TableType.REALTIME).setTableName(rawTableName).setTimeColumnName("timeColumn") .setNumReplicas(3).setStreamConfigs(getStreamConfigMap()).build(); - final LLCSegmentName seg1 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); - final LLCSegmentName seg2 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); - final LLCSegmentName seg3 = new LLCSegmentName(rawTableName, 2, 1, System.currentTimeMillis()); - IdealState idealState = new IdealState(tableName); + + LLCSegmentName seg1 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); + LLCSegmentName seg2 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); + LLCSegmentName seg3 = new LLCSegmentName(rawTableName, 2, 1, System.currentTimeMillis()); + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState(seg1.getSegmentName(), "pinot1", "ONLINE"); idealState.setPartitionState(seg1.getSegmentName(), "pinot2", "ONLINE"); idealState.setPartitionState(seg1.getSegmentName(), "pinot3", "ONLINE"); @@ -204,7 +196,7 @@ public void realtimeBasicTest() idealState.setReplicas("3"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(realtimeTableName); externalView.setState(seg1.getSegmentName(), "pinot1", "ONLINE"); externalView.setState(seg1.getSegmentName(), "pinot2", "ONLINE"); externalView.setState(seg1.getSegmentName(), "pinot3", "ONLINE"); @@ -218,13 +210,11 @@ public void realtimeBasicTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableConfig(realtimeTableName)).thenReturn(tableConfig); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(externalView); ZNRecord znRecord = new ZNRecord("0"); znRecord.setSimpleField(CommonConstants.Segment.Realtime.END_OFFSET, "10000"); when(_helixPropertyStore.get(anyString(), any(), anyInt())).thenReturn(znRecord); @@ -251,27 +241,25 @@ public void realtimeBasicTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.REPLICATION_FROM_CONFIG), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.REPLICATION_FROM_CONFIG), 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.MISSING_CONSUMING_SEGMENT_TOTAL_COUNT), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.MISSING_CONSUMING_SEGMENT_TOTAL_COUNT), 2); } Map getStreamConfigMap() { - return ImmutableMap.of( - "streamType", "kafka", - "stream.kafka.consumer.type", "simple", - "stream.kafka.topic.name", "test", + return Map.of("streamType", "kafka", "stream.kafka.consumer.type", "simple", "stream.kafka.topic.name", "test", "stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", "stream.kafka.consumer.factory.class.name", "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); @@ -281,8 +269,7 @@ Map getStreamConfigMap() { public void missingEVPartitionTest() throws Exception { String offlineTableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(offlineTableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); @@ -317,21 +304,19 @@ public void missingEVPartitionTest() ZkHelixPropertyStore propertyStore; { propertyStore = (ZkHelixPropertyStore) mock(ZkHelixPropertyStore.class); - when(propertyStore.get("/SEGMENTS/myTable_OFFLINE/myTable_3", null, AccessOption.PERSISTENT)) - .thenReturn(znrecord); + when(propertyStore.get("/SEGMENTS/myTable_OFFLINE/myTable_3", null, AccessOption.PERSISTENT)).thenReturn( + znrecord); } { _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_3")) - .thenReturn(new SegmentZKMetadata(znrecord)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_3")).thenReturn( + new SegmentZKMetadata(znrecord)); } { _config = mock(ControllerConf.class); @@ -355,25 +340,25 @@ public void missingEVPartitionTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 75); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 1111); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 75); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 1111); } @Test public void missingEVTest() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + String realtimeTableName = "myTable_REALTIME"; + + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot3", "ONLINE"); @@ -388,11 +373,9 @@ public void missingEVTest() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -416,30 +399,28 @@ public void missingEVTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void missingIdealTest() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList<>(); - allTableNames.add(tableName); + String realtimeTableName = "myTable_REALTIME"; { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(null); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(null); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -463,24 +444,24 @@ public void missingIdealTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, + ControllerGauge.SEGMENTS_IN_ERROR_STATE)); + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.TABLE_COMPRESSED_SIZE)); + assertFalse( + MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS)); + assertFalse( + MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS)); + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, + ControllerGauge.TABLE_COMPRESSED_SIZE)); } @Test public void missingEVPartitionPushTest() throws Exception { String offlineTableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(offlineTableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_1", "pinot1", "ONLINE"); @@ -527,15 +508,13 @@ public void missingEVPartitionPushTest() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_0")) - .thenReturn(new SegmentZKMetadata(znrecord)); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_2")) - .thenReturn(new SegmentZKMetadata(znrecord2)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_0")).thenReturn( + new SegmentZKMetadata(znrecord)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_2")).thenReturn( + new SegmentZKMetadata(znrecord2)); } { _config = mock(ControllerConf.class); @@ -559,27 +538,27 @@ public void missingEVPartitionPushTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void noReplicas() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + String realtimeTableName = "myTable_REALTIME"; + + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState("myTable_0", "pinot1", "OFFLINE"); idealState.setPartitionState("myTable_0", "pinot2", "OFFLINE"); idealState.setPartitionState("myTable_0", "pinot3", "OFFLINE"); @@ -590,11 +569,9 @@ public void noReplicas() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -618,26 +595,26 @@ public void noReplicas() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + 1); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS), + 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); } @Test - public void disabledTableTest() - throws Exception { + public void disabledTableTest() { + String offlineTableName = "myTable_OFFLINE"; - final String tableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); // disable table in idealstate idealState.enable(false); idealState.setPartitionState("myTable_OFFLINE", "pinot1", "OFFLINE"); @@ -648,11 +625,9 @@ public void disabledTableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -669,23 +644,21 @@ public void disabledTableTest() _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, _executorService); + // verify state before test - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 0); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 0); + // update metrics _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); } @Test - public void disabledEmptyTableTest() - throws Exception { + public void disabledEmptyTableTest() { + String offlineTableName = "myTable_OFFLINE"; - final String tableName = "myTable_OFFLINE"; - List allTableNames = Lists.newArrayList(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); // disable table in idealstate idealState.enable(false); idealState.setReplicas("1"); @@ -693,11 +666,9 @@ public void disabledEmptyTableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -714,14 +685,14 @@ public void disabledEmptyTableTest() _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, _executorService); + // verify state before test - Assert.assertFalse( - MetricValueUtils.globalGaugeExists(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT)); + assertFalse(MetricValueUtils.globalGaugeExists(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT)); + // update metrics _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); } @Test @@ -734,22 +705,20 @@ public void noSegments() @Test public void lessThanOnePercentSegmentsUnavailableTest() - throws Exception { - String tableName = "myTable_OFFLINE"; - int numSegments = 200; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + throws Exception { + String offlineTableName = "myTable_OFFLINE"; TableConfig tableConfig = - new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setNumReplicas(1).build(); + new TableConfigBuilder(TableType.OFFLINE).setTableName(offlineTableName).setNumReplicas(1).build(); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); + int numSegments = 200; for (int i = 0; i < numSegments; i++) { idealState.setPartitionState("myTable_" + i, "pinot1", "ONLINE"); } idealState.setReplicas("1"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(offlineTableName); externalView.setState("myTable_0", "pinot1", "OFFLINE"); for (int i = 1; i < numSegments; i++) { externalView.setState("myTable_" + i, "pinot1", "ONLINE"); @@ -757,19 +726,17 @@ public void lessThanOnePercentSegmentsUnavailableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableConfig(offlineTableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); } { _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - SegmentLineage segmentLineage = new SegmentLineage(tableName); - when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + tableName), any(), eq(AccessOption.PERSISTENT))) - .thenReturn(segmentLineage.toZNRecord()); + SegmentLineage segmentLineage = new SegmentLineage(offlineTableName); + when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + offlineTableName), any(), + eq(AccessOption.PERSISTENT))).thenReturn(segmentLineage.toZNRecord()); } { _config = mock(ControllerConf.class); @@ -788,37 +755,35 @@ public void lessThanOnePercentSegmentsUnavailableTest() _metricsRegistry = PinotMetricUtils.getPinotMetricsRegistry(); _controllerMetrics = new ControllerMetrics(_metricsRegistry); _segmentStatusChecker = - new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, - _executorService); + new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, + _executorService); _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 99); + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 99); } public void noSegmentsInternal(final int nReplicas) throws Exception { - final String tableName = "myTable_REALTIME"; + String realtimeTableName = "myTable_REALTIME"; + String nReplicasStr = Integer.toString(nReplicas); int nReplicasExpectedValue = nReplicas; if (nReplicas < 0) { nReplicasStr = "abc"; nReplicasExpectedValue = 1; } - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(realtimeTableName); idealState.setReplicas(nReplicasStr); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -843,15 +808,17 @@ public void noSegmentsInternal(final int nReplicas) _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), nReplicasExpectedValue); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + nReplicasExpectedValue); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS), + 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); } } diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java index 15c3cf6d8197..f4e0eb46b14f 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java @@ -19,7 +19,6 @@ package org.apache.pinot.controller.helix.core.periodictask; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; @@ -35,7 +34,6 @@ import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -89,9 +87,7 @@ public void processTable(String tableNameWithType) { public void beforeTest() { List tables = new ArrayList<>(_numTables); IntStream.range(0, _numTables).forEach(i -> tables.add("table_" + i + " _OFFLINE")); - when(_resourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_resourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(tables); + when(_resourceManager.getAllTables()).thenReturn(tables); when(_leadControllerManager.isLeaderForTable(anyString())).thenReturn(true); } @@ -109,7 +105,6 @@ public void testRandomInitialDelay() { _task.getInitialDelayInSeconds() >= ControllerConf.ControllerPeriodicTasksConf.MIN_INITIAL_DELAY_IN_SECONDS); assertTrue( _task.getInitialDelayInSeconds() < ControllerConf.ControllerPeriodicTasksConf.MAX_INITIAL_DELAY_IN_SECONDS); - assertEquals(_task.getIntervalInSeconds(), RUN_FREQUENCY_IN_SECONDS); } @@ -124,7 +119,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); // Run periodic task with leadership resetState(); @@ -133,8 +128,7 @@ public void testControllerPeriodicTaskCalls() { assertTrue(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), _numTables); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), - _numTables); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); @@ -145,7 +139,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), 0); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); assertTrue(_stopTaskCalled.get()); assertFalse(_task.isStarted()); @@ -156,7 +150,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), 0); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); assertFalse(_stopTaskCalled.get()); assertFalse(_task.isStarted()); @@ -169,7 +163,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); // Run periodic task with leadership resetState(); @@ -178,7 +172,7 @@ public void testControllerPeriodicTaskCalls() { assertTrue(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), _numTables); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); } diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java index ce5e31e5ef13..b3e656de9ead 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java @@ -19,7 +19,6 @@ package org.apache.pinot.controller.helix.core.retention; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -43,15 +42,13 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.mockito.ArgumentMatchers; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; public class RetentionManagerTest { @@ -60,8 +57,7 @@ public class RetentionManagerTest { private static final String OFFLINE_TABLE_NAME = TableNameBuilder.OFFLINE.tableNameWithType(TEST_TABLE_NAME); private static final String REALTIME_TABLE_NAME = TableNameBuilder.REALTIME.tableNameWithType(TEST_TABLE_NAME); - private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long dayAfterTomorrowTimeStamp) - throws Exception { + private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long dayAfterTomorrowTimeStamp) { List segmentsZKMetadata = new ArrayList<>(); // Create metadata for 10 segments really old, that will be removed by the retention manager. final int numOlderSegments = 10; @@ -105,8 +101,7 @@ private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long } @Test - public void testRetentionWithMinutes() - throws Exception { + public void testRetentionWithMinutes() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long minutesSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60; final long pastMinutesSinceEpoch = 22383360L; @@ -114,8 +109,7 @@ public void testRetentionWithMinutes() } @Test - public void testRetentionWithSeconds() - throws Exception { + public void testRetentionWithSeconds() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long secondsSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60 * 60; final long pastSecondsSinceEpoch = 1343001600L; @@ -123,8 +117,7 @@ public void testRetentionWithSeconds() } @Test - public void testRetentionWithMillis() - throws Exception { + public void testRetentionWithMillis() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long millisSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60 * 60 * 1000; final long pastMillisSinceEpoch = 1343001600000L; @@ -132,8 +125,7 @@ public void testRetentionWithMillis() } @Test - public void testRetentionWithHours() - throws Exception { + public void testRetentionWithHours() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long hoursSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24; final long pastHoursSinceEpoch = 373056L; @@ -141,8 +133,7 @@ public void testRetentionWithHours() } @Test - public void testRetentionWithDays() - throws Exception { + public void testRetentionWithDays() { final long daysSinceEpochTimeStamp = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long pastDaysSinceEpoch = 15544L; testDifferentTimeUnits(pastDaysSinceEpoch, TimeUnit.DAYS, daysSinceEpochTimeStamp); @@ -161,10 +152,8 @@ private TableConfig createRealtimeTableConfig1(int replicaCount) { private void setupPinotHelixResourceManager(TableConfig tableConfig, final List removedSegments, PinotHelixResourceManager resourceManager, LeadControllerManager leadControllerManager) { - final String tableNameWithType = tableConfig.getTableName(); - when(resourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(resourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(Collections.singletonList(tableNameWithType)); + String tableNameWithType = tableConfig.getTableName(); + when(resourceManager.getAllTables()).thenReturn(List.of(tableNameWithType)); ZkHelixPropertyStore propertyStore = mock(ZkHelixPropertyStore.class); when(resourceManager.getPropertyStore()).thenReturn(propertyStore); @@ -172,38 +161,27 @@ private void setupPinotHelixResourceManager(TableConfig tableConfig, final List< SegmentDeletionManager deletionManager = mock(SegmentDeletionManager.class); // Ignore the call to SegmentDeletionManager.removeAgedDeletedSegments. we only test that the call is made once per // run of the retention manager - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) - throws Throwable { - return null; - } - }).when(deletionManager).removeAgedDeletedSegments(leadControllerManager); + doAnswer(invocationOnMock -> null).when(deletionManager).removeAgedDeletedSegments(leadControllerManager); when(resourceManager.getSegmentDeletionManager()).thenReturn(deletionManager); // If and when PinotHelixResourceManager.deleteSegments() is invoked, make sure that the segments deleted // are exactly the same as the ones we expect to be deleted. - doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) - throws Throwable { - Object[] args = invocationOnMock.getArguments(); - String tableNameArg = (String) args[0]; - Assert.assertEquals(tableNameArg, tableNameWithType); - List segmentListArg = (List) args[1]; - Assert.assertEquals(segmentListArg.size(), removedSegments.size()); - for (String segmentName : removedSegments) { - Assert.assertTrue(segmentListArg.contains(segmentName)); - } - return null; + doAnswer(invocationOnMock -> { + Object[] args = invocationOnMock.getArguments(); + String tableNameArg = (String) args[0]; + assertEquals(tableNameArg, tableNameWithType); + List segmentListArg = (List) args[1]; + assertEquals(segmentListArg.size(), removedSegments.size()); + for (String segmentName : removedSegments) { + assertTrue(segmentListArg.contains(segmentName)); } - }).when(resourceManager).deleteSegments(anyString(), ArgumentMatchers.anyList()); + return null; + }).when(resourceManager).deleteSegments(anyString(), anyList()); } // This test makes sure that we clean up the segments marked OFFLINE in realtime for more than 7 days @Test - public void testRealtimeLLCCleanup() - throws Exception { + public void testRealtimeLLCCleanup() { final int initialNumSegments = 8; final long now = System.currentTimeMillis(); @@ -237,8 +215,7 @@ public void testRealtimeLLCCleanup() // This test makes sure that we do not clean up last llc completed segments @Test - public void testRealtimeLastLLCCleanup() - throws Exception { + public void testRealtimeLastLLCCleanup() { final long now = System.currentTimeMillis(); final int replicaCount = 1; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index d4b770222f17..c46a85690dc5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -63,6 +63,8 @@ import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.index.loader.LoaderUtils; import org.apache.pinot.segment.spi.ImmutableSegment; +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoader; @@ -433,6 +435,14 @@ public Map getSegmentErrors() { } } + @Override + public List getSegmentContexts(List selectedSegments, + Map queryOptions) { + List segmentContexts = new ArrayList<>(selectedSegments.size()); + selectedSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + return segmentContexts; + } + @Override public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingConfig, SegmentZKMetadata zkMetadata, SegmentMetadata localMetadata, @Nullable Schema schema, boolean forceDownload) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index e226a13502a2..d15e79116052 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -92,7 +92,9 @@ import org.apache.pinot.spi.stream.StreamDataDecoderImpl; import org.apache.pinot.spi.stream.StreamDataDecoderResult; import org.apache.pinot.spi.stream.StreamDecoderProvider; +import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageDecoder; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -242,7 +244,7 @@ public void deleteSegmentFile() { private final PartitionUpsertMetadataManager _partitionUpsertMetadataManager; private final BooleanSupplier _isReadyToConsumeData; private final MutableSegmentImpl _realtimeSegment; - private volatile StreamPartitionMsgOffset _currentOffset; + private volatile StreamPartitionMsgOffset _currentOffset; // Next offset to be consumed private volatile State _state; private volatile int _numRowsConsumed = 0; private volatile int _numRowsIndexed = 0; // Can be different from _numRowsConsumed when metrics update is enabled. @@ -262,7 +264,7 @@ public void deleteSegmentFile() { private volatile boolean _hasMessagesFetched = false; private volatile boolean _endOfPartitionGroup = false; private volatile boolean _forceCommitMessageReceived = false; - private StreamPartitionMsgOffset _finalOffset; // Used when we want to catch up to this one + private volatile StreamPartitionMsgOffset _finalOffset; // Exclusive, used when we want to catch up to this one private volatile boolean _shouldStop = false; // It takes 30s to locate controller leader, and more if there are multiple controller failures. @@ -426,8 +428,7 @@ protected boolean consumeLoop() // Update _currentOffset upon return from this method MessageBatch messageBatch; try { - messageBatch = - _partitionGroupConsumer.fetchMessages(_currentOffset, null, _streamConfig.getFetchTimeoutMillis()); + messageBatch = _partitionGroupConsumer.fetchMessages(_currentOffset, _streamConfig.getFetchTimeoutMillis()); //track realtime rows fetched on a table level. This included valid + invalid rows _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FETCHED, messageBatch.getUnfilteredMessageCount()); @@ -519,13 +520,13 @@ protected boolean consumeLoop() } /** - * @param messagesAndOffsets batch of messages to process + * @param messageBatch batch of messages to process * @param idlePipeSleepTimeMillis wait time in case no messages were read * @return returns true if the process loop ended before processing the batch, false * otherwise */ - private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePipeSleepTimeMillis) { - int messageCount = messagesAndOffsets.getMessageCount(); + private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSleepTimeMillis) { + int messageCount = messageBatch.getMessageCount(); _partitionRateLimiter.throttle(messageCount); _serverRateLimiter.throttle(messageCount); @@ -536,11 +537,9 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi int indexedMessageCount = 0; int streamMessageCount = 0; boolean canTakeMore = true; - boolean hasTransformedRows = false; TransformPipeline.Result reusedResult = new TransformPipeline.Result(); boolean prematureExit = false; - RowMetadata msgMetadata = null; for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); @@ -573,9 +572,19 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi } // Decode message - StreamDataDecoderResult decodedRow = _streamDataDecoder.decode(messagesAndOffsets.getStreamMessage(index)); - msgMetadata = messagesAndOffsets.getStreamMessage(index).getMetadata(); - StreamPartitionMsgOffset messageOffset = messagesAndOffsets.getNextStreamPartitionMsgOffsetAtIndex(index); + StreamMessage streamMessage = messageBatch.getStreamMessage(index); + StreamDataDecoderResult decodedRow = _streamDataDecoder.decode(streamMessage); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + StreamPartitionMsgOffset offset = null; + StreamPartitionMsgOffset nextOffset = null; + if (metadata != null) { + offset = metadata.getOffset(); + nextOffset = metadata.getNextOffset(); + } + // Backward compatible + if (nextOffset == null) { + nextOffset = messageBatch.getNextStreamPartitionMsgOffsetAtIndex(index); + } if (decodedRow.getException() != null) { // TODO: based on a config, decide whether the record should be silently dropped or stop further consumption on // decode error @@ -591,15 +600,15 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi _numRowsErrored++; // when exception happens we prefer abandoning the whole batch and not partially indexing some rows reusedResult.getTransformedRows().clear(); - String errorMessage = String.format("Caught exception while transforming the record at offset: %s , row: %s", - messageOffset, decodedRow.getResult()); + String errorMessage = + String.format("Caught exception while transforming the record at offset: %s , row: %s", offset, + decodedRow.getResult()); _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); } if (reusedResult.getSkippedRowCount() > 0) { - realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, - reusedResult.getSkippedRowCount(), realtimeRowsDroppedMeter); + realtimeRowsDroppedMeter = _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, + reusedResult.getSkippedRowCount(), realtimeRowsDroppedMeter); } if (reusedResult.getIncompleteRowCount() > 0) { realtimeIncompleteRowsConsumedMeter = @@ -607,14 +616,11 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi reusedResult.getIncompleteRowCount(), realtimeIncompleteRowsConsumedMeter); } List transformedRows = reusedResult.getTransformedRows(); - if (transformedRows.size() > 0) { - hasTransformedRows = true; - } for (GenericRow transformedRow : transformedRows) { try { - canTakeMore = _realtimeSegment.index(transformedRow, msgMetadata); + canTakeMore = _realtimeSegment.index(transformedRow, metadata); indexedMessageCount++; - _lastRowMetadata = msgMetadata; + _lastRowMetadata = metadata; _lastConsumedTimestampMs = System.currentTimeMillis(); realtimeRowsConsumedMeter = _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_CONSUMED, 1, @@ -622,36 +628,24 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_ROWS_CONSUMED, 1L); } catch (Exception e) { _numRowsErrored++; - String errorMessage = String.format("Caught exception while indexing the record at offset: %s , row: %s", - messageOffset, transformedRow); + String errorMessage = + String.format("Caught exception while indexing the record at offset: %s , row: %s", offset, + transformedRow); _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); } } } - _currentOffset = messageOffset; + _currentOffset = nextOffset; _numRowsIndexed = _realtimeSegment.getNumDocsIndexed(); _numRowsConsumed++; streamMessageCount++; } - if (indexedMessageCount > 0) { - // Record Ingestion delay for this partition with metadata for last message we processed - updateIngestionDelay(_lastRowMetadata); - } else if (!hasTransformedRows && (msgMetadata != null)) { - // If all messages were filtered by transformation, we still attempt to update ingestion delay using - // the metadata for the last message we processed if any. - updateIngestionDelay(msgMetadata); - } - updateCurrentDocumentCountMetrics(); - if (messagesAndOffsets.getUnfilteredMessageCount() > 0) { + if (messageBatch.getUnfilteredMessageCount() > 0) { + updateIngestionDelay(messageBatch.getLastMessageMetadata()); _hasMessagesFetched = true; - if (messageCount == 0) { - // If we received events from the stream but all were filtered, we attempt to estimate the ingestion - // delay from the metadata of the last filtered message received. - updateIngestionDelay(messagesAndOffsets.getLastMessageMetadata()); - } if (streamMessageCount > 0 && _segmentLogger.isDebugEnabled()) { _segmentLogger.debug("Indexed {} messages ({} messages read from stream) current offset {}", indexedMessageCount, streamMessageCount, _currentOffset); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 0c62ab9b4d73..8e50049028e9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -46,6 +47,7 @@ import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.SegmentUtils; import org.apache.pinot.common.utils.TarGzCompressionUtils; +import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; import org.apache.pinot.core.data.manager.BaseTableDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; @@ -67,6 +69,7 @@ import org.apache.pinot.segment.local.utils.tablestate.TableStateUtils; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.DedupConfig; import org.apache.pinot.spi.config.table.IndexingConfig; @@ -300,6 +303,17 @@ public void onConsumingToOnline(String segmentNameStr) { _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId()); } + @Override + public List getSegmentContexts(List selectedSegments, + Map queryOptions) { + List segmentContexts = new ArrayList<>(selectedSegments.size()); + selectedSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + if (isUpsertEnabled() && !QueryOptionsUtils.isSkipUpsert(queryOptions)) { + _tableUpsertMetadataManager.setSegmentContexts(segmentContexts); + } + return segmentContexts; + } + /** * Returns all partitionGroupIds for the partitions hosted by this server for current table. * @apiNote this involves Zookeeper read and should not be used frequently due to efficiency concerns. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java index b803526882ad..0ef9bb22071d 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java @@ -29,7 +29,7 @@ import org.apache.pinot.core.operator.combine.BaseCombineOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider; import org.apache.pinot.spi.exception.EarlyTerminationException; import org.apache.pinot.spi.exception.QueryCancelledException; @@ -40,15 +40,15 @@ public class InstanceResponseOperator extends BaseOperator _combineOperator; - protected final List _indexSegments; + protected final List _segmentContexts; protected final List _fetchContexts; protected final int _fetchContextSize; protected final QueryContext _queryContext; - public InstanceResponseOperator(BaseCombineOperator combineOperator, List indexSegments, + public InstanceResponseOperator(BaseCombineOperator combineOperator, List segmentContexts, List fetchContexts, QueryContext queryContext) { _combineOperator = combineOperator; - _indexSegments = indexSegments; + _segmentContexts = segmentContexts; _fetchContexts = fetchContexts; _fetchContextSize = fetchContexts.size(); _queryContext = queryContext; @@ -128,13 +128,13 @@ private BaseResultsBlock getCombinedResults() { public void prefetchAll() { for (int i = 0; i < _fetchContextSize; i++) { - _indexSegments.get(i).prefetch(_fetchContexts.get(i)); + _segmentContexts.get(i).getIndexSegment().prefetch(_fetchContexts.get(i)); } } public void releaseAll() { for (int i = 0; i < _fetchContextSize; i++) { - _indexSegments.get(i).release(_fetchContexts.get(i)); + _segmentContexts.get(i).getIndexSegment().release(_fetchContexts.get(i)); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java index ef503ae7fff1..61a5b8cc83d9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java @@ -30,7 +30,7 @@ import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.exception.EarlyTerminationException; import org.apache.pinot.spi.exception.QueryCancelledException; import org.apache.pinot.spi.trace.Tracing; @@ -42,9 +42,10 @@ public class StreamingInstanceResponseOperator extends InstanceResponseOperator private final BaseStreamingCombineOperator _streamingCombineOperator; private final ResultsBlockStreamer _streamer; - public StreamingInstanceResponseOperator(BaseCombineOperator combinedOperator, List indexSegments, - List fetchContexts, ResultsBlockStreamer streamer, QueryContext queryContext) { - super(combinedOperator, indexSegments, fetchContexts, queryContext); + public StreamingInstanceResponseOperator(BaseCombineOperator combinedOperator, + List segmentContexts, List fetchContexts, ResultsBlockStreamer streamer, + QueryContext queryContext) { + super(combinedOperator, segmentContexts, fetchContexts, queryContext); _streamingCombineOperator = combinedOperator instanceof BaseStreamingCombineOperator ? (BaseStreamingCombineOperator) combinedOperator : null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java index 73406c20fa60..0d9ed602f8ab 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java @@ -20,7 +20,7 @@ import org.apache.pinot.core.operator.AcquireReleaseColumnsSegmentOperator; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -36,13 +36,13 @@ public class AcquireReleaseColumnsSegmentPlanNode implements PlanNode { private final PlanNode _childPlanNode; - private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final FetchContext _fetchContext; - public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, IndexSegment indexSegment, + public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, SegmentContext segmentContext, FetchContext fetchContext) { _childPlanNode = childPlanNode; - _indexSegment = indexSegment; + _segmentContext = segmentContext; _fetchContext = fetchContext; } @@ -52,6 +52,6 @@ public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, IndexSegment */ @Override public AcquireReleaseColumnsSegmentOperator run() { - return new AcquireReleaseColumnsSegmentOperator(_childPlanNode, _indexSegment, _fetchContext); + return new AcquireReleaseColumnsSegmentOperator(_childPlanNode, _segmentContext.getIndexSegment(), _fetchContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java index d09e465cf048..2a1321f1b978 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java @@ -34,6 +34,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.AggregationFunctionType; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import static org.apache.pinot.segment.spi.AggregationFunctionType.*; @@ -56,10 +57,12 @@ public class AggregationPlanNode implements PlanNode { EnumSet.of(COUNT, MIN, MINMV, MAX, MAXMV, MINMAXRANGE, MINMAXRANGEMV); private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public AggregationPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public AggregationPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -74,7 +77,7 @@ public Operator run() { */ private FilteredAggregationOperator buildFilteredAggOperator() { return new FilteredAggregationOperator(_queryContext, - AggregationFunctionUtils.buildFilteredAggregationInfos(_indexSegment, _queryContext), + AggregationFunctionUtils.buildFilteredAggregationInfos(_segmentContext, _queryContext), _indexSegment.getSegmentMetadata().getTotalDocs()); } @@ -88,7 +91,7 @@ public Operator buildNonFilteredAggOperator() { assert aggregationFunctions != null; int numTotalDocs = _indexSegment.getSegmentMetadata().getTotalDocs(); - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, _queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(_segmentContext, _queryContext); BaseFilterOperator filterOperator = filterPlanNode.run(); if (!_queryContext.isNullHandlingEnabled()) { @@ -110,7 +113,7 @@ public Operator buildNonFilteredAggOperator() { } AggregationInfo aggregationInfo = - AggregationFunctionUtils.buildAggregationInfo(_indexSegment, _queryContext, aggregationFunctions, + AggregationFunctionUtils.buildAggregationInfo(_segmentContext, _queryContext, aggregationFunctions, _queryContext.getFilter(), filterOperator, filterPlanNode.getPredicateEvaluators()); return new AggregationOperator(_queryContext, aggregationInfo, numTotalDocs); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java index 0ae7e3f31dc6..44ffcd98aa70 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java @@ -27,6 +27,7 @@ import org.apache.pinot.core.operator.query.DistinctOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; @@ -36,10 +37,12 @@ */ public class DistinctPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public DistinctPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public DistinctPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -70,7 +73,7 @@ public Operator run() { } BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new DistinctOperator(_indexSegment, _queryContext, projectOperator); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java index 766d80e74dba..0e3558014b58 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java @@ -22,22 +22,21 @@ import org.apache.pinot.core.operator.DocIdSetOperator; import org.apache.pinot.core.operator.filter.BaseFilterOperator; import org.apache.pinot.core.query.request.context.QueryContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class DocIdSetPlanNode implements PlanNode { public static final int MAX_DOC_PER_CALL = 10_000; - - private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final int _maxDocPerCall; private final BaseFilterOperator _filterOperator; - public DocIdSetPlanNode(IndexSegment indexSegment, QueryContext queryContext, int maxDocPerCall, + public DocIdSetPlanNode(SegmentContext segmentContext, QueryContext queryContext, int maxDocPerCall, @Nullable BaseFilterOperator filterOperator) { assert maxDocPerCall > 0 && maxDocPerCall <= MAX_DOC_PER_CALL; - _indexSegment = indexSegment; + _segmentContext = segmentContext; _queryContext = queryContext; _maxDocPerCall = maxDocPerCall; _filterOperator = filterOperator; @@ -46,7 +45,7 @@ public DocIdSetPlanNode(IndexSegment indexSegment, QueryContext queryContext, in @Override public DocIdSetOperator run() { return new DocIdSetOperator( - _filterOperator != null ? _filterOperator : new FilterPlanNode(_indexSegment, _queryContext).run(), + _filterOperator != null ? _filterOperator : new FilterPlanNode(_segmentContext, _queryContext).run(), _maxDocPerCall); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java index 9a6d17cc9f30..156624922093 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java @@ -53,8 +53,8 @@ import org.apache.pinot.segment.local.realtime.impl.invertedindex.NativeMutableTextIndex; import org.apache.pinot.segment.local.segment.index.readers.text.NativeTextIndexReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; -import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; import org.apache.pinot.segment.spi.index.reader.JsonIndexReader; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; import org.apache.pinot.segment.spi.index.reader.TextIndexReader; @@ -66,50 +66,40 @@ public class FilterPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final FilterContext _filter; // Cache the predicate evaluators private final List> _predicateEvaluators = new ArrayList<>(4); - public FilterPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - this(indexSegment, queryContext, null); + public FilterPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + this(segmentContext, queryContext, null); } - public FilterPlanNode(IndexSegment indexSegment, QueryContext queryContext, @Nullable FilterContext filter) { - _indexSegment = indexSegment; + public FilterPlanNode(SegmentContext segmentContext, QueryContext queryContext, @Nullable FilterContext filter) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; _filter = filter != null ? filter : _queryContext.getFilter(); } @Override public BaseFilterOperator run() { - // NOTE: Snapshot the queryableDocIds before reading the numDocs to prevent the latest updates getting lost - MutableRoaringBitmap queryableDocIdSnapshot = null; - if (!_queryContext.isSkipUpsert()) { - ThreadSafeMutableRoaringBitmap queryableDocIds = _indexSegment.getQueryableDocIds(); - if (queryableDocIds != null) { - queryableDocIdSnapshot = queryableDocIds.getMutableRoaringBitmap(); - } else { - ThreadSafeMutableRoaringBitmap validDocIds = _indexSegment.getValidDocIds(); - if (validDocIds != null) { - queryableDocIdSnapshot = validDocIds.getMutableRoaringBitmap(); - } - } - } + MutableRoaringBitmap queryableDocIdsSnapshot = _segmentContext.getQueryableDocIdsSnapshot(); int numDocs = _indexSegment.getSegmentMetadata().getTotalDocs(); if (_filter != null) { BaseFilterOperator filterOperator = constructPhysicalOperator(_filter, numDocs); - if (queryableDocIdSnapshot != null) { - BaseFilterOperator validDocFilter = new BitmapBasedFilterOperator(queryableDocIdSnapshot, false, numDocs); + if (queryableDocIdsSnapshot != null) { + BaseFilterOperator validDocFilter = new BitmapBasedFilterOperator(queryableDocIdsSnapshot, false, numDocs); return FilterOperatorUtils.getAndFilterOperator(_queryContext, Arrays.asList(filterOperator, validDocFilter), numDocs); } else { return filterOperator; } - } else if (queryableDocIdSnapshot != null) { - return new BitmapBasedFilterOperator(queryableDocIdSnapshot, false, numDocs); + } else if (queryableDocIdsSnapshot != null) { + return new BitmapBasedFilterOperator(queryableDocIdsSnapshot, false, numDocs); } else { return new MatchAllFilterOperator(numDocs); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java index 89b1afa5524d..b44b8dc8ee4f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -33,10 +34,12 @@ */ public class GroupByPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public GroupByPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public GroupByPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -48,15 +51,15 @@ public Operator run() { private FilteredGroupByOperator buildFilteredGroupByPlan() { return new FilteredGroupByOperator(_queryContext, - AggregationFunctionUtils.buildFilteredAggregationInfos(_indexSegment, _queryContext), + AggregationFunctionUtils.buildFilteredAggregationInfos(_segmentContext, _queryContext), _indexSegment.getSegmentMetadata().getTotalDocs()); } private GroupByOperator buildNonFilteredGroupByPlan() { - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, _queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(_segmentContext, _queryContext); BaseFilterOperator filterOperator = filterPlanNode.run(); AggregationFunctionUtils.AggregationInfo aggregationInfo = - AggregationFunctionUtils.buildAggregationInfo(_indexSegment, _queryContext, + AggregationFunctionUtils.buildAggregationInfo(_segmentContext, _queryContext, _queryContext.getAggregationFunctions(), _queryContext.getFilter(), filterOperator, filterPlanNode.getPredicateEvaluators()); return new GroupByOperator(_queryContext, aggregationInfo, _indexSegment.getSegmentMetadata().getTotalDocs()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java index 57b0c18f9e9c..bb600d0c40d9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java @@ -22,25 +22,25 @@ import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class InstanceResponsePlanNode implements PlanNode { protected final CombinePlanNode _combinePlanNode; - protected final List _indexSegments; + protected final List _segmentContexts; protected final List _fetchContexts; protected final QueryContext _queryContext; - public InstanceResponsePlanNode(CombinePlanNode combinePlanNode, List indexSegments, + public InstanceResponsePlanNode(CombinePlanNode combinePlanNode, List segmentContexts, List fetchContexts, QueryContext queryContext) { _combinePlanNode = combinePlanNode; - _indexSegments = indexSegments; + _segmentContexts = segmentContexts; _fetchContexts = fetchContexts; _queryContext = queryContext; } @Override public InstanceResponseOperator run() { - return new InstanceResponseOperator(_combinePlanNode.run(), _indexSegments, _fetchContexts, _queryContext); + return new InstanceResponseOperator(_combinePlanNode.run(), _segmentContexts, _fetchContexts, _queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java index 1654a47b1006..6bc763009084 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java @@ -34,6 +34,7 @@ import org.apache.pinot.core.operator.transform.TransformOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; @@ -42,23 +43,25 @@ */ public class ProjectPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final Collection _expressions; private final int _maxDocsPerCall; private final BaseFilterOperator _filterOperator; - public ProjectPlanNode(IndexSegment indexSegment, QueryContext queryContext, + public ProjectPlanNode(SegmentContext segmentContext, QueryContext queryContext, Collection expressions, int maxDocsPerCall, @Nullable BaseFilterOperator filterOperator) { - _indexSegment = indexSegment; + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; _expressions = expressions; _maxDocsPerCall = maxDocsPerCall; _filterOperator = filterOperator; } - public ProjectPlanNode(IndexSegment indexSegment, QueryContext queryContext, + public ProjectPlanNode(SegmentContext segmentContext, QueryContext queryContext, Collection expressions, int maxDocsPerCall) { - this(indexSegment, queryContext, expressions, maxDocsPerCall, null); + this(segmentContext, queryContext, expressions, maxDocsPerCall, null); } @Override @@ -75,8 +78,8 @@ public BaseProjectOperator run() { projectionColumns.forEach(column -> dataSourceMap.put(column, _indexSegment.getDataSource(column))); // NOTE: Skip creating DocIdSetOperator when maxDocsPerCall is 0 (for selection query with LIMIT 0) DocIdSetOperator docIdSetOperator = - _maxDocsPerCall > 0 ? new DocIdSetPlanNode(_indexSegment, _queryContext, _maxDocsPerCall, _filterOperator).run() - : null; + _maxDocsPerCall > 0 ? new DocIdSetPlanNode(_segmentContext, _queryContext, _maxDocsPerCall, + _filterOperator).run() : null; ProjectionOperator projectionOperator = ProjectionOperatorUtils.getProjectionOperator(dataSourceMap, docIdSetOperator); return hasNonIdentifierExpression ? new TransformOperator(_queryContext, projectionOperator, _expressions) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java index 395b4f2ffaf6..e936cd694996 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java @@ -35,6 +35,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.selection.SelectionOperatorUtils; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; @@ -44,10 +45,12 @@ */ public class SelectionPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public SelectionPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public SelectionPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -58,7 +61,8 @@ public Operator run() { if (limit == 0) { // Empty selection (LIMIT 0) - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressions, 0).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(_segmentContext, _queryContext, expressions, 0).run(); return new EmptySelectionOperator(_indexSegment, _queryContext, expressions, projectOperator); } @@ -68,7 +72,7 @@ public Operator run() { // ie: SELECT ... FROM Table WHERE ... LIMIT 10 int maxDocsPerCall = Math.min(limit, DocIdSetPlanNode.MAX_DOC_PER_CALL); BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionOnlyOperator(_indexSegment, _queryContext, expressions, projectOperator); } int numOrderByExpressions = orderByExpressions.size(); @@ -87,12 +91,12 @@ public Operator run() { maxDocsPerCall = Math.min(limit + _queryContext.getOffset(), DocIdSetPlanNode.MAX_DOC_PER_CALL); } BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionPartiallyOrderedByAscOperator(_indexSegment, _queryContext, expressions, projectOperator, sortedColumnsPrefixSize); } else { BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionPartiallyOrderedByDescOperation(_indexSegment, _queryContext, expressions, projectOperator, sortedColumnsPrefixSize); } @@ -101,7 +105,7 @@ public Operator run() { // All output expressions are ordered // ie: SELECT not_sorted1, not_sorted2 FROM Table WHERE ... ORDER BY not_sorted1, not_sorted2 LIMIT 10 OFFSET 5 BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new SelectionOrderByOperator(_indexSegment, _queryContext, expressions, projectOperator); } // Not all output expressions are ordered, only fetch the order-by expressions and docId to avoid the @@ -111,7 +115,7 @@ public Operator run() { for (OrderByExpressionContext orderByExpression : orderByExpressions) { expressionsToTransform.add(orderByExpression.getExpression()); } - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressionsToTransform, + BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentContext, _queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new SelectionOrderByOperator(_indexSegment, _queryContext, expressions, projectOperator); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java index 274ca45faf15..52276fa09719 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java @@ -24,21 +24,21 @@ import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class StreamingInstanceResponsePlanNode extends InstanceResponsePlanNode { private final ResultsBlockStreamer _streamer; - public StreamingInstanceResponsePlanNode(CombinePlanNode combinePlanNode, List indexSegments, + public StreamingInstanceResponsePlanNode(CombinePlanNode combinePlanNode, List segmentContexts, List fetchContexts, QueryContext queryContext, ResultsBlockStreamer streamer) { - super(combinePlanNode, indexSegments, fetchContexts, queryContext); + super(combinePlanNode, segmentContexts, fetchContexts, queryContext); _streamer = streamer; } @Override public InstanceResponseOperator run() { - return new StreamingInstanceResponseOperator(_combinePlanNode.run(), _indexSegments, _fetchContexts, _streamer, + return new StreamingInstanceResponseOperator(_combinePlanNode.run(), _segmentContexts, _fetchContexts, _streamer, _queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java index 94ab5eef1504..386dff2591ae 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.selection.SelectionOperatorUtils; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -35,19 +36,21 @@ */ public class StreamingSelectionPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public StreamingSelectionPlanNode(IndexSegment indexSegment, QueryContext queryContext) { + public StreamingSelectionPlanNode(SegmentContext segmentContext, QueryContext queryContext) { Preconditions.checkState(queryContext.getOrderByExpressions() == null, "Selection order-by is not supported for streaming"); - _indexSegment = indexSegment; + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @Override public StreamingSelectionOnlyOperator run() { List expressions = SelectionOperatorUtils.extractExpressions(_queryContext, _indexSegment); - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressions, + BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentContext, _queryContext, expressions, Math.min(_queryContext.getLimit(), DocIdSetPlanNode.MAX_DOC_PER_CALL)).run(); return new StreamingSelectionOnlyOperator(_indexSegment, _queryContext, expressions, projectOperator); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java index bf565e68e6f6..54cb034377a5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java @@ -53,6 +53,7 @@ import org.apache.pinot.core.util.GroupByUtils; import org.apache.pinot.segment.spi.FetchContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.env.PinotConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,34 +134,33 @@ public void init(PinotConfiguration queryExecutorConfig) { _minServerGroupTrimSize, _groupByTrimThreshold); } - @Override - public Plan makeInstancePlan(List indexSegments, QueryContext queryContext, + public Plan makeInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ServerMetrics serverMetrics) { applyQueryOptions(queryContext); - int numSegments = indexSegments.size(); + int numSegments = segmentContexts.size(); List planNodes = new ArrayList<>(numSegments); List fetchContexts; - if (queryContext.isEnablePrefetch()) { fetchContexts = new ArrayList<>(numSegments); - for (IndexSegment indexSegment : indexSegments) { - FetchContext fetchContext = _fetchPlanner.planFetchForProcessing(indexSegment, queryContext); + for (SegmentContext segmentContext : segmentContexts) { + FetchContext fetchContext = + _fetchPlanner.planFetchForProcessing(segmentContext.getIndexSegment(), queryContext); fetchContexts.add(fetchContext); planNodes.add( - new AcquireReleaseColumnsSegmentPlanNode(makeSegmentPlanNode(indexSegment, queryContext), indexSegment, + new AcquireReleaseColumnsSegmentPlanNode(makeSegmentPlanNode(segmentContext, queryContext), segmentContext, fetchContext)); } } else { fetchContexts = Collections.emptyList(); - for (IndexSegment indexSegment : indexSegments) { - planNodes.add(makeSegmentPlanNode(indexSegment, queryContext)); + for (SegmentContext segmentContext : segmentContexts) { + planNodes.add(makeSegmentPlanNode(segmentContext, queryContext)); } } CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, executorService, null); return new GlobalPlanImplV0( - new InstanceResponsePlanNode(combinePlanNode, indexSegments, fetchContexts, queryContext)); + new InstanceResponsePlanNode(combinePlanNode, segmentContexts, fetchContexts, queryContext)); } private void applyQueryOptions(QueryContext queryContext) { @@ -231,47 +231,45 @@ private void applyQueryOptions(QueryContext queryContext) { } @Override - public PlanNode makeSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - rewriteQueryContextWithHints(queryContext, indexSegment); + public PlanNode makeSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + rewriteQueryContextWithHints(queryContext, segmentContext.getIndexSegment()); if (QueryContextUtils.isAggregationQuery(queryContext)) { List groupByExpressions = queryContext.getGroupByExpressions(); if (groupByExpressions != null) { // Group-by query - return new GroupByPlanNode(indexSegment, queryContext); + return new GroupByPlanNode(segmentContext, queryContext); } else { // Aggregation query - return new AggregationPlanNode(indexSegment, queryContext); + return new AggregationPlanNode(segmentContext, queryContext); } } else if (QueryContextUtils.isSelectionQuery(queryContext)) { - return new SelectionPlanNode(indexSegment, queryContext); + return new SelectionPlanNode(segmentContext, queryContext); } else { assert QueryContextUtils.isDistinctQuery(queryContext); - return new DistinctPlanNode(indexSegment, queryContext); + return new DistinctPlanNode(segmentContext, queryContext); } } - @Override - public Plan makeStreamingInstancePlan(List indexSegments, QueryContext queryContext, + public Plan makeStreamingInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ResultsBlockStreamer streamer, ServerMetrics serverMetrics) { applyQueryOptions(queryContext); - - List planNodes = new ArrayList<>(indexSegments.size()); - for (IndexSegment indexSegment : indexSegments) { - planNodes.add(makeStreamingSegmentPlanNode(indexSegment, queryContext)); + List planNodes = new ArrayList<>(segmentContexts.size()); + for (SegmentContext segmentContext : segmentContexts) { + planNodes.add(makeStreamingSegmentPlanNode(segmentContext, queryContext)); } CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, executorService, streamer); return new GlobalPlanImplV0( - new StreamingInstanceResponsePlanNode(combinePlanNode, indexSegments, Collections.emptyList(), queryContext, + new StreamingInstanceResponsePlanNode(combinePlanNode, segmentContexts, Collections.emptyList(), queryContext, streamer)); } @Override - public PlanNode makeStreamingSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext) { + public PlanNode makeStreamingSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext) { if (QueryContextUtils.isSelectionOnlyQuery(queryContext) && queryContext.getLimit() != 0) { // Use streaming operator only for non-empty selection-only query - return new StreamingSelectionPlanNode(indexSegment, queryContext); + return new StreamingSelectionPlanNode(segmentContext, queryContext); } else { - return makeSegmentPlanNode(indexSegment, queryContext); + return makeSegmentPlanNode(segmentContext, queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java index b5f8da91c335..36e62bd86735 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java @@ -25,7 +25,7 @@ import org.apache.pinot.core.plan.PlanNode; import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.annotations.InterfaceAudience; import org.apache.pinot.spi.env.PinotConfiguration; @@ -44,24 +44,24 @@ public interface PlanMaker { /** * Returns an instance level {@link Plan} which contains the logical execution plan for multiple segments. */ - Plan makeInstancePlan(List indexSegments, QueryContext queryContext, ExecutorService executorService, - ServerMetrics serverMetrics); + Plan makeInstancePlan(List segmentContexts, QueryContext queryContext, + ExecutorService executorService, ServerMetrics serverMetrics); /** * Returns a segment level {@link PlanNode} which contains the logical execution plan for one segment. */ - PlanNode makeSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext); + PlanNode makeSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext); /** * Returns an instance level {@link Plan} for a streaming query which contains the logical execution plan for multiple * segments. */ - Plan makeStreamingInstancePlan(List indexSegments, QueryContext queryContext, + Plan makeStreamingInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ResultsBlockStreamer streamer, ServerMetrics serverMetrics); /** * Returns a segment level {@link PlanNode} for a streaming query which contains the logical execution plan for one * segment. */ - PlanNode makeStreamingSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext); + PlanNode makeStreamingSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java index 48e5d4784cf3..8d6cbf4aac6f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java @@ -47,7 +47,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.startree.StarTreeUtils; import org.apache.pinot.segment.spi.AggregationFunctionType; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair; @@ -243,16 +243,15 @@ public boolean isUseStarTree() { /** * Builds {@link AggregationInfo} for aggregations. */ - public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, QueryContext queryContext, + public static AggregationInfo buildAggregationInfo(SegmentContext segmentContext, QueryContext queryContext, AggregationFunction[] aggregationFunctions, @Nullable FilterContext filter, BaseFilterOperator filterOperator, List> predicateEvaluators) { BaseProjectOperator projectOperator = null; // TODO: Create a short-circuit ProjectOperator when filter result is empty if (!filterOperator.isResultEmpty()) { - projectOperator = - StarTreeUtils.createStarTreeBasedProjectOperator(indexSegment, queryContext, aggregationFunctions, filter, - predicateEvaluators); + projectOperator = StarTreeUtils.createStarTreeBasedProjectOperator(segmentContext.getIndexSegment(), queryContext, + aggregationFunctions, filter, predicateEvaluators); } if (projectOperator != null) { @@ -262,7 +261,7 @@ public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, Qu AggregationFunctionUtils.collectExpressionsToTransform(aggregationFunctions, queryContext.getGroupByExpressions()); projectOperator = - new ProjectPlanNode(indexSegment, queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL, + new ProjectPlanNode(segmentContext, queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL, filterOperator).run(); return new AggregationInfo(aggregationFunctions, projectOperator, false); } @@ -271,11 +270,11 @@ public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, Qu /** * Builds swim-lanes (list of {@link AggregationInfo}) for filtered aggregations. */ - public static List buildFilteredAggregationInfos(IndexSegment indexSegment, + public static List buildFilteredAggregationInfos(SegmentContext segmentContext, QueryContext queryContext) { assert queryContext.getAggregationFunctions() != null && queryContext.getFilteredAggregationFunctions() != null; - FilterPlanNode mainFilterPlan = new FilterPlanNode(indexSegment, queryContext); + FilterPlanNode mainFilterPlan = new FilterPlanNode(segmentContext, queryContext); BaseFilterOperator mainFilterOperator = mainFilterPlan.run(); List> mainPredicateEvaluators = mainFilterPlan.getPredicateEvaluators(); @@ -285,7 +284,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i Set expressions = collectExpressionsToTransform(aggregationFunctions, queryContext.getGroupByExpressions()); BaseProjectOperator projectOperator = - new ProjectPlanNode(indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL, + new ProjectPlanNode(segmentContext, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL, mainFilterOperator).run(); return Collections.singletonList(new AggregationInfo(aggregationFunctions, projectOperator, false)); } @@ -307,7 +306,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i combinedFilter = FilterContext.forAnd(List.of(mainFilter, filter)); } - FilterPlanNode subFilterPlan = new FilterPlanNode(indexSegment, queryContext, filter); + FilterPlanNode subFilterPlan = new FilterPlanNode(segmentContext, queryContext, filter); BaseFilterOperator subFilterOperator = subFilterPlan.run(); BaseFilterOperator combinedFilterOperator; if (mainFilterOperator.isResultMatchingAll() || subFilterOperator.isResultEmpty()) { @@ -342,7 +341,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i AggregationFunction[] aggregationFunctions = filteredAggregationContext._aggregationFunctions.toArray(new AggregationFunction[0]); aggregationInfos.add( - buildAggregationInfo(indexSegment, queryContext, aggregationFunctions, filteredAggregationContext._filter, + buildAggregationInfo(segmentContext, queryContext, aggregationFunctions, filteredAggregationContext._filter, filteredAggregationContext._filterOperator, filteredAggregationContext._predicateEvaluators)); } } @@ -350,7 +349,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i if (!nonFilteredFunctions.isEmpty()) { AggregationFunction[] aggregationFunctions = nonFilteredFunctions.toArray(new AggregationFunction[0]); aggregationInfos.add( - buildAggregationInfo(indexSegment, queryContext, aggregationFunctions, mainFilter, mainFilterOperator, + buildAggregationInfo(segmentContext, queryContext, aggregationFunctions, mainFilter, mainFilterOperator, mainPredicateEvaluators)); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index b8c5383bbb69..82664cd57ffe 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -70,6 +70,7 @@ import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.MutableSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.env.PinotConfiguration; @@ -256,8 +257,9 @@ private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, E InstanceResponseBlock instanceResponse = null; try { - instanceResponse = executeInternal(indexSegments, queryContext, timerContext, executorService, streamer, - queryRequest.isEnableStreaming()); + instanceResponse = + executeInternal(tableDataManager, indexSegments, queryContext, timerContext, executorService, streamer, + queryRequest.isEnableStreaming()); } catch (Exception e) { _serverMetrics.addMeteredTableValue(tableNameWithType, ServerMeter.QUERY_EXECUTION_EXCEPTIONS, 1); instanceResponse = new InstanceResponseBlock(); @@ -344,11 +346,11 @@ private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, E } // NOTE: This method might change indexSegments. Do not use it after calling this method. - private InstanceResponseBlock executeInternal(List indexSegments, QueryContext queryContext, - TimerContext timerContext, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer, - boolean enableStreaming) + private InstanceResponseBlock executeInternal(TableDataManager tableDataManager, List indexSegments, + QueryContext queryContext, TimerContext timerContext, ExecutorService executorService, + @Nullable ResultsBlockStreamer streamer, boolean enableStreaming) throws Exception { - handleSubquery(queryContext, indexSegments, timerContext, executorService); + handleSubquery(queryContext, tableDataManager, indexSegments, timerContext, executorService); // Compute total docs for the table before pruning the segments long numTotalDocs = 0; @@ -379,10 +381,12 @@ private InstanceResponseBlock executeInternal(List indexSegments, } } else { TimerContext.Timer planBuildTimer = timerContext.startNewPhaseTimer(ServerQueryPhase.BUILD_QUERY_PLAN); + List selectedSegmentContexts = + tableDataManager.getSegmentContexts(selectedSegments, queryContext.getQueryOptions()); Plan queryPlan = - enableStreaming ? _planMaker.makeStreamingInstancePlan(selectedSegments, queryContext, executorService, + enableStreaming ? _planMaker.makeStreamingInstancePlan(selectedSegmentContexts, queryContext, executorService, streamer, _serverMetrics) - : _planMaker.makeInstancePlan(selectedSegments, queryContext, executorService, _serverMetrics); + : _planMaker.makeInstancePlan(selectedSegmentContexts, queryContext, executorService, _serverMetrics); planBuildTimer.stopAndRecord(); TimerContext.Timer planExecTimer = timerContext.startNewPhaseTimer(ServerQueryPhase.QUERY_PLAN_EXECUTION); @@ -523,12 +527,13 @@ public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryCon * Handles the subquery in the given query. *

Currently only supports subquery within the filter. */ - private void handleSubquery(QueryContext queryContext, List indexSegments, TimerContext timerContext, - ExecutorService executorService) + private void handleSubquery(QueryContext queryContext, TableDataManager tableDataManager, + List indexSegments, TimerContext timerContext, ExecutorService executorService) throws Exception { FilterContext filter = queryContext.getFilter(); if (filter != null && !filter.isConstant()) { - handleSubquery(filter, indexSegments, timerContext, executorService, queryContext.getEndTimeMs()); + handleSubquery(filter, tableDataManager, indexSegments, timerContext, executorService, + queryContext.getEndTimeMs()); } } @@ -536,16 +541,17 @@ private void handleSubquery(QueryContext queryContext, List indexS * Handles the subquery in the given filter. *

Currently only supports subquery within the lhs of the predicate. */ - private void handleSubquery(FilterContext filter, List indexSegments, TimerContext timerContext, - ExecutorService executorService, long endTimeMs) + private void handleSubquery(FilterContext filter, TableDataManager tableDataManager, List indexSegments, + TimerContext timerContext, ExecutorService executorService, long endTimeMs) throws Exception { List children = filter.getChildren(); if (children != null) { for (FilterContext child : children) { - handleSubquery(child, indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(child, tableDataManager, indexSegments, timerContext, executorService, endTimeMs); } } else { - handleSubquery(filter.getPredicate().getLhs(), indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(filter.getPredicate().getLhs(), tableDataManager, indexSegments, timerContext, executorService, + endTimeMs); } } @@ -556,8 +562,8 @@ private void handleSubquery(FilterContext filter, List indexSegmen *

Currently only supports ID_SET subquery within the IN_PARTITIONED_SUBQUERY transform function, which will be * rewritten to an IN_ID_SET transform function. */ - private void handleSubquery(ExpressionContext expression, List indexSegments, TimerContext timerContext, - ExecutorService executorService, long endTimeMs) + private void handleSubquery(ExpressionContext expression, TableDataManager tableDataManager, + List indexSegments, TimerContext timerContext, ExecutorService executorService, long endTimeMs) throws Exception { FunctionContext function = expression.getFunction(); if (function == null) { @@ -584,7 +590,8 @@ private void handleSubquery(ExpressionContext expression, List ind subquery.setEndTimeMs(endTimeMs); // Make a clone of indexSegments because the method might modify the list InstanceResponseBlock instanceResponse = - executeInternal(new ArrayList<>(indexSegments), subquery, timerContext, executorService, null, false); + executeInternal(tableDataManager, new ArrayList<>(indexSegments), subquery, timerContext, executorService, + null, false); BaseResultsBlock resultsBlock = instanceResponse.getResultsBlock(); Preconditions.checkState(resultsBlock instanceof AggregationResultsBlock, "Got unexpected results block type: %s, expecting aggregation results", @@ -598,7 +605,7 @@ private void handleSubquery(ExpressionContext expression, List ind ExpressionContext.forLiteralContext(FieldSpec.DataType.STRING, ((IdSet) result).toBase64String())); } else { for (ExpressionContext argument : arguments) { - handleSubquery(argument, indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(argument, tableDataManager, indexSegments, timerContext, executorService, endTimeMs); } } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java index 0b94197b6ade..ef3c0e1e540b 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java @@ -38,6 +38,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -240,7 +241,7 @@ private SelectionResultsBlock getCombineResult(String query) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); List planNodes = new ArrayList<>(NUM_SEGMENTS); for (IndexSegment indexSegment : _indexSegments) { - planNodes.add(PLAN_MAKER.makeSegmentPlanNode(indexSegment, queryContext)); + planNodes.add(PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(indexSegment), queryContext)); } queryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, EXECUTOR, null); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java index 7dddf57bc90d..a99c0412a3b2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java @@ -36,6 +36,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -131,8 +132,9 @@ public void testPartialSortNullWithoutNullHandling() { private List executeQuery(QueryContext queryContext) { List expressions = SelectionOperatorUtils.extractExpressions(queryContext, _segmentWithNullValues); - BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentWithNullValues, queryContext, expressions, - DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(new SegmentContext(_segmentWithNullValues), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); SelectionOrderByOperator operator = new SelectionOrderByOperator( _segmentWithNullValues, diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java index fb583146ca9b..d21978b1ee4e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java @@ -35,6 +35,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -76,8 +77,9 @@ public void testNullHandling() { queryContext.setNullHandlingEnabled(true); List expressions = SelectionOperatorUtils.extractExpressions(queryContext, _segmentWithNullValues); - BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentWithNullValues, queryContext, expressions, - DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(new SegmentContext(_segmentWithNullValues), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); StreamingSelectionOnlyOperator operator = new StreamingSelectionOnlyOperator( _segmentWithNullValues, diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java index b123f2a807a3..2fe671fa293b 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.Constants; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; import org.mockito.stubbing.Answer; @@ -48,8 +49,8 @@ public void testConsistentSnapshot() when(segment.getValidDocIds()).thenReturn(bitmap); AtomicInteger numDocs = new AtomicInteger(0); when(meta.getTotalDocs()).then((Answer) invocationOnMock -> numDocs.get()); - QueryContext ctx = mock(QueryContext.class); - when(ctx.getFilter()).thenReturn(null); + QueryContext queryContext = mock(QueryContext.class); + when(queryContext.getFilter()).thenReturn(null); numDocs.set(3); bitmap.add(0); @@ -69,14 +70,16 @@ public void testConsistentSnapshot() // Result should be invariant - always exactly 3 docs for (int i = 0; i < 10_000; i++) { - assertEquals(getNumberOfFilteredDocs(segment, ctx), 3); + SegmentContext segmentContext = new SegmentContext(segment); + segmentContext.setQueryableDocIdsSnapshot(TestUtils.getQueryableDocIdsSnapshotFromSegment(segment)); + assertEquals(getNumberOfFilteredDocs(segmentContext, queryContext), 3); } updater.join(); } - private int getNumberOfFilteredDocs(IndexSegment segment, QueryContext ctx) { - FilterPlanNode node = new FilterPlanNode(segment, ctx); + private int getNumberOfFilteredDocs(SegmentContext segmentContext, QueryContext queryContext) { + FilterPlanNode node = new FilterPlanNode(segmentContext, queryContext); BaseFilterOperator op = node.run(); int numDocsFiltered = 0; FilterBlock block = op.nextBlock(); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java new file mode 100644 index 000000000000..171489e5edf4 --- /dev/null +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java @@ -0,0 +1,43 @@ +/** + * 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.pinot.core.plan; + +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; +import org.roaringbitmap.buffer.MutableRoaringBitmap; + + +public class TestUtils { + private TestUtils() { + } + + public static MutableRoaringBitmap getQueryableDocIdsSnapshotFromSegment(IndexSegment segment) { + MutableRoaringBitmap queryableDocIdsSnapshot = null; + ThreadSafeMutableRoaringBitmap queryableDocIds = segment.getQueryableDocIds(); + if (queryableDocIds != null) { + queryableDocIdsSnapshot = queryableDocIds.getMutableRoaringBitmap(); + } else { + ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds(); + if (validDocIds != null) { + queryableDocIdsSnapshot = validDocIds.getMutableRoaringBitmap(); + } + } + return queryableDocIdsSnapshot; + } +} diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java index cf3e33f9388d..f066b1618b45 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java @@ -32,6 +32,7 @@ import org.apache.pinot.core.operator.query.GroupByOperator; import org.apache.pinot.core.operator.query.NonScanBasedAggregationOperator; import org.apache.pinot.core.operator.query.SelectionOnlyOperator; +import org.apache.pinot.core.plan.TestUtils; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; @@ -40,6 +41,7 @@ import org.apache.pinot.segment.local.upsert.ConcurrentMapPartitionUpsertMetadataManager; import org.apache.pinot.segment.local.upsert.UpsertContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; import org.apache.pinot.segment.spi.index.StandardIndexes; @@ -153,9 +155,12 @@ public void deleteSegment() { public void testPlanMaker(String query, Class> operatorClass, Class> upsertOperatorClass) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); - Operator operator = PLAN_MAKER.makeSegmentPlanNode(_indexSegment, queryContext).run(); + Operator operator = PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(_indexSegment), queryContext).run(); assertTrue(operatorClass.isInstance(operator)); - Operator upsertOperator = PLAN_MAKER.makeSegmentPlanNode(_upsertIndexSegment, queryContext).run(); + + SegmentContext segmentContext = new SegmentContext(_upsertIndexSegment); + segmentContext.setQueryableDocIdsSnapshot(TestUtils.getQueryableDocIdsSnapshotFromSegment(_upsertIndexSegment)); + Operator upsertOperator = PLAN_MAKER.makeSegmentPlanNode(segmentContext, queryContext).run(); assertTrue(upsertOperatorClass.isInstance(upsertOperator)); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java index b1eb471c704e..68a180ea886d 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java @@ -78,10 +78,5 @@ String expectedAggrWithoutNull75(Scenario scenario) { String expectedAggrWithoutNull90(Scenario scenario) { return "7.100000000000001"; } - - @Override - String expectedAggrWithoutNull100(Scenario scenario) { - return super.expectedAggrWithoutNull100(scenario); - } } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java index 5e2ae8af3413..79a60bde987a 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -151,8 +152,8 @@ private void setup() for (String column : MV_COLUMNS) { expressions.add(ExpressionContext.forIdentifier(column)); } - ProjectPlanNode projectPlanNode = - new ProjectPlanNode(indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL); + ProjectPlanNode projectPlanNode = new ProjectPlanNode(new SegmentContext(indexSegment), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL); _projectOperator = projectPlanNode.run(); _valueBlock = _projectOperator.nextBlock(); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java index dba3faefe6c2..1cebefe52a11 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java @@ -41,6 +41,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; @@ -119,7 +120,8 @@ void testGroupByTrim(QueryContext queryContext, int minSegmentGroupTrimSize, int queryContext.setMinServerGroupTrimSize(minServerGroupTrimSize); // Create a query operator - Operator groupByOperator = new GroupByPlanNode(_indexSegment, queryContext).run(); + Operator groupByOperator = + new GroupByPlanNode(new SegmentContext(_indexSegment), queryContext).run(); GroupByCombineOperator combineOperator = new GroupByCombineOperator(Collections.singletonList(groupByOperator), queryContext, _executorService); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java index 790f062dc703..fa2882c192b2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -154,8 +155,8 @@ public void setUp() for (String column : COLUMNS) { expressions.add(ExpressionContext.forIdentifier(column)); } - ProjectPlanNode projectPlanNode = - new ProjectPlanNode(_indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL); + ProjectPlanNode projectPlanNode = new ProjectPlanNode(new SegmentContext(_indexSegment), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL); _projectOperator = projectPlanNode.run(); _valueBlock = _projectOperator.nextBlock(); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java index cbb77066f2a8..3f52ca4740f2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeoutException; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; @@ -34,7 +33,7 @@ import org.apache.pinot.plugin.inputformat.avro.AvroUtils; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; @@ -44,15 +43,14 @@ /** - * Implementation of {@link PartitionLevelConsumer} for fake stream + * Implementation of {@link PartitionGroupConsumer} for fake stream * Unpacks tar files in /resources/data/On_Time_Performance_2014_partition_.tar.gz as source of messages */ -public class FakePartitionLevelConsumer implements PartitionLevelConsumer { +public class FakePartitionLevelConsumer implements PartitionGroupConsumer { + private static final Logger LOGGER = LoggerFactory.getLogger(FakePartitionLevelConsumer.class); - private static final Logger LOGGER = LoggerFactory.getLogger(FakePartitionLevelConsumer.class.getName()); - - private List _messageOffsets = new ArrayList<>(); - private List _messageBytes = new ArrayList<>(); + private final List _messageOffsets = new ArrayList<>(); + private final List _messageBytes = new ArrayList<>(); private final int _defaultBatchSize; FakePartitionLevelConsumer(int partition, StreamConfig streamConfig, int defaultBatchSize) { @@ -95,32 +93,15 @@ public class FakePartitionLevelConsumer implements PartitionLevelConsumer { } } - public MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) - throws TimeoutException { - throw new UnsupportedOperationException("This method is deprecated"); - } - @Override - public MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, - int timeoutMillis) - throws TimeoutException { - if (startOffset.compareTo(FakeStreamConfigUtils.getLargestOffset()) >= 0) { - return new FakeStreamMessageBatch(Collections.emptyList(), Collections.emptyList()); - } - if (startOffset.compareTo(FakeStreamConfigUtils.getSmallestOffset()) < 0) { - startOffset = FakeStreamConfigUtils.getSmallestOffset(); - } - if (endOffset == null || endOffset.compareTo(FakeStreamConfigUtils.getLargestOffset()) > 0) { - endOffset = FakeStreamConfigUtils.getLargestOffset(); - } + public MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { int startOffsetInt = (int) ((LongMsgOffset) startOffset).getOffset(); - int endOffsetInt = (int) ((LongMsgOffset) endOffset).getOffset(); - if (endOffsetInt > _messageOffsets.size() && _defaultBatchSize > 0) { - // Hack to get multiple batches - endOffsetInt = startOffsetInt + _defaultBatchSize; + if (startOffsetInt >= _messageOffsets.size()) { + return new FakeStreamMessageBatch(Collections.emptyList(), Collections.emptyList(), startOffsetInt); } - return new FakeStreamMessageBatch(_messageOffsets.subList(startOffsetInt, endOffsetInt), - _messageBytes.subList(startOffsetInt, endOffsetInt)); + int endOffsetInt = Math.min(startOffsetInt + _defaultBatchSize, _messageOffsets.size()); + return new FakeStreamMessageBatch(_messageBytes.subList(startOffsetInt, endOffsetInt), + _messageOffsets.subList(startOffsetInt, endOffsetInt), endOffsetInt); } @Override diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java index 008d0dd9001d..3888a8cfe5c0 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java @@ -18,21 +18,10 @@ */ package org.apache.pinot.core.realtime.impl.fakestream; -import org.apache.pinot.segment.local.utils.IngestionUtils; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; -import org.apache.pinot.spi.stream.StreamDecoderProvider; -import org.apache.pinot.spi.stream.StreamMessageDecoder; import org.apache.pinot.spi.stream.StreamMetadataProvider; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; /** @@ -43,11 +32,6 @@ */ public class FakeStreamConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new FakePartitionLevelConsumer(partition, _streamConfig, FakeStreamConfigUtils.MESSAGE_BATCH_SIZE); - } - @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new FakeStreamMetadataProvider(_streamConfig); @@ -58,43 +42,10 @@ public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return new FakeStreamMetadataProvider(_streamConfig); } - public static void main(String[] args) - throws Exception { - String clientId = "client_id_localhost_tester"; - - // stream config - int numPartitions = 5; - StreamConfig streamConfig = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs(numPartitions); - - // stream consumer factory - StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - - // stream metadata provider - StreamMetadataProvider streamMetadataProvider = streamConsumerFactory.createStreamMetadataProvider(clientId); - int partitionCount = streamMetadataProvider.fetchPartitionCount(10_000); - System.out.println(partitionCount); - - // Partition metadata provider - int partition = 3; - StreamMetadataProvider partitionMetadataProvider = - streamConsumerFactory.createPartitionMetadataProvider(clientId, partition); - StreamPartitionMsgOffset partitionOffset = - partitionMetadataProvider.fetchStreamPartitionOffset(OffsetCriteria.SMALLEST_OFFSET_CRITERIA, 10_000); - System.out.println(partitionOffset); - - // Partition level consumer - PartitionLevelConsumer partitionLevelConsumer = - streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); - MessageBatch messageBatch = - partitionLevelConsumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(40), 10_000); - - // Message decoder - Schema pinotSchema = FakeStreamConfigUtils.getPinotSchema(); - TableConfig tableConfig = FakeStreamConfigUtils.getTableConfig(); - StreamMessageDecoder streamMessageDecoder = StreamDecoderProvider.create(streamConfig, - IngestionUtils.getFieldsForRecordExtractor(tableConfig.getIngestionConfig(), pinotSchema)); - GenericRow decodedRow = new GenericRow(); - streamMessageDecoder.decode(messageBatch.getMessageAtIndex(0), decodedRow); - System.out.println(decodedRow); + @Override + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + return new FakePartitionLevelConsumer(partitionGroupConsumptionStatus.getPartitionGroupId(), _streamConfig, + FakeStreamConfigUtils.MESSAGE_BATCH_SIZE); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java index 134aac9fd84a..41cb158e63f6 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java @@ -19,50 +19,43 @@ package org.apache.pinot.core.realtime.impl.fakestream; import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; /** * MessageBatch implementation for the fake stream */ -public class FakeStreamMessageBatch implements MessageBatch { - private List _messageOffsets; - private List _messageBytes; - - FakeStreamMessageBatch(List messageOffsets, List messageBytes) { - _messageOffsets = messageOffsets; - _messageBytes = messageBytes; +class FakeStreamMessageBatch implements MessageBatch { + private final List _values; + private final List _offsets; + private final int _offsetOfNextBatch; + + FakeStreamMessageBatch(List values, List offsets, int offsetOfNextBatch) { + _values = values; + _offsets = offsets; + _offsetOfNextBatch = offsetOfNextBatch; } @Override public int getMessageCount() { - return _messageOffsets.size(); - } - - @Override - public byte[] getMessageAtIndex(int index) { - return _messageBytes.get(index); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return _messageOffsets.get(index); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageBytes.get(index).length; + return _values.size(); } @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("This method is deprecated"); + public BytesStreamMessage getStreamMessage(int index) { + byte[] value = _values.get(index); + int offset = _offsets.get(index); + return new BytesStreamMessage(value, + new StreamMessageMetadata.Builder().setOffset(new LongMsgOffset(offset), new LongMsgOffset(offset + 1)) + .build()); } @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - return new LongMsgOffset(_messageOffsets.get(index) + 1); + public StreamPartitionMsgOffset getOffsetOfNextBatch() { + return new LongMsgOffset(_offsetOfNextBatch); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java index 14c148bbb271..d4e6d3da4694 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java @@ -47,6 +47,7 @@ import org.apache.pinot.segment.spi.AggregationFunctionType; import org.apache.pinot.segment.spi.Constants; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.Dictionary; @@ -230,7 +231,7 @@ public void tearDown() private void testUnsupportedFilter(String query) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); filterPlanNode.run(); Map> predicateEvaluatorsMap = StarTreeUtils.extractPredicateEvaluatorsMap(_indexSegment, queryContext.getFilter(), @@ -262,7 +263,7 @@ private void testQuery(String query) List groupByColumns = new ArrayList<>(groupByColumnSet); // Filter - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); filterPlanNode.run(); Map> predicateEvaluatorsMap = StarTreeUtils.extractPredicateEvaluatorsMap(_indexSegment, queryContext.getFilter(), @@ -285,7 +286,7 @@ private void testQuery(String query) computeStarTreeResult(starTreeFilterPlanNode, starTreeAggregationColumnReaders, starTreeGroupByColumnReaders); // Extract values without star-tree - FilterPlanNode nonStarTreeFilterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode nonStarTreeFilterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); List nonStarTreeAggregationColumnReaders = new ArrayList<>(numAggregations); List nonStarTreeAggregationColumnDictionaries = new ArrayList<>(numAggregations); for (AggregationFunctionColumnPair aggregationFunctionColumnPair : aggregationFunctionColumnPairs) { diff --git a/pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java index 7e9ca3e78e36..642f44464bae 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import it.unimi.dsi.fastutil.ints.Int2DoubleMap; import it.unimi.dsi.fastutil.ints.Int2DoubleOpenHashMap; @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Random; -import org.apache.pinot.core.util.IntDoubleIndexedPriorityQueue; import org.apache.pinot.spi.utils.Pairs; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java index e67e8d18cf0e..aeb51ff7c108 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import java.util.ArrayList; import java.util.Collections; @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Random; -import org.apache.pinot.core.util.IntObjectIndexedPriorityQueue; import org.apache.pinot.segment.local.customobject.AvgPair; import org.apache.pinot.spi.utils.Pairs; import org.testng.Assert; diff --git a/pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java index 92ac06f9677b..0c5c6ca555da 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java @@ -16,13 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pinot.core.util.QueryMultiThreadingUtils; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java index fa5a5068dfae..2746eeb8a1a8 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java @@ -19,6 +19,7 @@ package org.apache.pinot.queries; import java.io.File; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -49,6 +50,7 @@ import org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.store.SegmentDirectory; @@ -91,7 +93,7 @@ protected T getOperator(String query) { PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(query); PinotQuery serverPinotQuery = GapfillUtils.stripGapfill(pinotQuery); QueryContext queryContext = QueryContextConverterUtils.getQueryContext(serverPinotQuery); - return (T) PLAN_MAKER.makeSegmentPlanNode(getIndexSegment(), queryContext).run(); + return (T) PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(getIndexSegment()), queryContext).run(); } /** @@ -206,7 +208,8 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker // Server side serverQueryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); - Plan plan = planMaker.makeInstancePlan(getIndexSegments(), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan = + planMaker.makeInstancePlan(getSegmentContexts(getIndexSegments()), serverQueryContext, EXECUTOR_SERVICE, null); InstanceResponseBlock instanceResponse; try { instanceResponse = @@ -241,6 +244,12 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker return brokerResponse; } + private static List getSegmentContexts(List indexSegments) { + List segmentContexts = new ArrayList<>(indexSegments.size()); + indexSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + return segmentContexts; + } + /** * Run optimized query on multiple index segments. *

Use this to test the whole flow from server to broker. @@ -297,8 +306,10 @@ private BrokerResponseNative getBrokerResponseDistinctInstances(PinotQuery pinot List> instances = getDistinctInstances(); // Server side serverQueryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); - Plan plan1 = planMaker.makeInstancePlan(instances.get(0), serverQueryContext, EXECUTOR_SERVICE, null); - Plan plan2 = planMaker.makeInstancePlan(instances.get(1), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan1 = + planMaker.makeInstancePlan(getSegmentContexts(instances.get(0)), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan2 = + planMaker.makeInstancePlan(getSegmentContexts(instances.get(1)), serverQueryContext, EXECUTOR_SERVICE, null); InstanceResponseBlock instanceResponse1; try { diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java index 3e87d7446b83..e8913e0a1b56 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -165,7 +166,7 @@ protected T getOperator(String query, boolean enableNullHan PinotQuery serverPinotQuery = GapfillUtils.stripGapfill(pinotQuery); QueryContext queryContext = QueryContextConverterUtils.getQueryContext(serverPinotQuery); - return (T) PLAN_MAKER.makeSegmentPlanNode(getIndexSegment(), queryContext).run(); + return (T) PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(getIndexSegment()), queryContext).run(); } @Test(dataProvider = "nullHandling") diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java index 1f04d16d3b1e..cfb570d80e0e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java @@ -135,7 +135,7 @@ protected void buildSegment() .setIngestionConfig(new IngestionConfig(null, null, null, null, Arrays.asList(new TransformConfig(M1_V2, "Groovy({INT_COL1_V3 == null || " + "INT_COL1_V3 == Integer.MIN_VALUE ? INT_COL1 : INT_COL1_V3 }, INT_COL1, INT_COL1_V3)")), - null, null, null)) + null, null, null, null)) .build(); Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension(D1, FieldSpec.DataType.STRING) diff --git a/pinot-integration-tests/pom.xml b/pinot-integration-tests/pom.xml index 995e423c666d..4c8ebc702b5d 100644 --- a/pinot-integration-tests/pom.xml +++ b/pinot-integration-tests/pom.xml @@ -34,8 +34,8 @@ ${basedir}/.. - 0.2.19 - 1.17.3 + 0.2.23 + 1.19.7 diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java index 2389fe8ba68f..0cfbc51dbf18 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java @@ -47,6 +47,7 @@ import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.plugin.stream.kafka20.KafkaConsumerFactory; +import org.apache.pinot.plugin.stream.kafka20.KafkaMessageBatch; import org.apache.pinot.plugin.stream.kafka20.KafkaPartitionLevelConsumer; import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -55,11 +56,10 @@ import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.JsonUtils; @@ -463,8 +463,10 @@ public static void init(String helixClusterName, HelixAdmin helixAdmin, String t _helixClusterName = helixClusterName; _tableName = tableName; } + @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { /* * The segment data manager is creating a consumer to consume rows into a segment. * Check the partition and sequence number of the segment and decide whether it @@ -473,6 +475,7 @@ public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int * - Throwing exception during consumption. * Make sure that this still works if retries are added in RealtimeSegmentDataManager */ + int partition = partitionGroupConsumptionStatus.getPartitionGroupId(); boolean exceptionDuringConsume = false; int seqNum = getSegmentSeqNum(partition); if (partition == PARTITION_FOR_EXCEPTIONS) { @@ -504,20 +507,21 @@ private int getSegmentSeqNum(int partition) { return seqNum.get(); } - public class ExceptingKafkaConsumer extends KafkaPartitionLevelConsumer { + public static class ExceptingKafkaConsumer extends KafkaPartitionLevelConsumer { private final boolean _exceptionDuringConsume; + public ExceptingKafkaConsumer(String clientId, StreamConfig streamConfig, int partition, boolean exceptionDuringConsume) { super(clientId, streamConfig, partition); _exceptionDuringConsume = exceptionDuringConsume; } + @Override - public MessageBatch> fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { + public KafkaMessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { if (_exceptionDuringConsume) { throw new RuntimeException("TestException during consumption"); } - return super.fetchMessages(startMsgOffset, endMsgOffset, timeoutMillis); + return super.fetchMessages(startOffset, timeoutMs); } } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java index 88ee0f400e38..ff41c76b1337 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java @@ -398,7 +398,7 @@ public void testStWithinQuery(boolean useMultiStageQueryEngine) JsonNode rows = pinotResponse.get("resultTable").get("rows"); for (int i = 0; i < rows.size(); i++) { JsonNode row = rows.get(i); - boolean actualResult = row.get(0).intValue() == 1 ? true : false; + boolean actualResult = row.get(0).intValue() == 1; boolean expectedResult = row.get(1).booleanValue(); Assert.assertEquals(actualResult, expectedResult); } diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java index de591f60cf10..4ed28c746f56 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java @@ -36,6 +36,7 @@ import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.FSTType; import org.apache.pinot.spi.config.table.FieldConfig; @@ -171,7 +172,7 @@ private void buildSegment(FSTType fstType) @Benchmark @CompilerControl(CompilerControl.Mode.DONT_INLINE) public void query(Blackhole bh) { - Operator operator = _planMaker.makeSegmentPlanNode(_indexSegment, _queryContext).run(); + Operator operator = _planMaker.makeSegmentPlanNode(new SegmentContext(_indexSegment), _queryContext).run(); bh.consume(operator); for (int i = 0; i < _numBlocks; i++) { bh.consume(operator.nextBlock()); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java index 2b28b6320367..649cc8ebb416 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java @@ -39,6 +39,7 @@ import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.FSTType; import org.apache.pinot.spi.config.table.FieldConfig; @@ -204,7 +205,7 @@ private ImmutableSegment loadSegment(FSTType fstType) @Benchmark @CompilerControl(CompilerControl.Mode.DONT_INLINE) public void query(Blackhole bh) { - Operator operator = _planMaker.makeSegmentPlanNode(_indexSegment, _queryContext).run(); + Operator operator = _planMaker.makeSegmentPlanNode(new SegmentContext(_indexSegment), _queryContext).run(); for (int i = 0; i < _numBlocks; i++) { bh.consume(operator.nextBlock()); } diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml index fd36cd868ce9..1dfba13c999b 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml @@ -36,7 +36,7 @@ ${basedir}/../../.. package - 3.5.0 + 3.5.1 diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index a34ada8eed3c..30d7cbc051be 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -39,12 +39,12 @@ com.azure azure-storage-file-datalake - 12.18.2 + 12.18.3 com.azure azure-identity - 1.11.3 + 1.12.0 @@ -52,22 +52,22 @@ com.azure azure-core-http-netty - 1.14.0 + 1.14.2 com.azure azure-core - 1.45.1 + 1.48.0 io.projectreactor reactor-core - 3.6.2 + 3.6.5 net.java.dev.jna jna-platform - 5.6.0 + 5.14.0 net.java.dev.jna @@ -77,7 +77,7 @@ com.microsoft.azure msal4j - 1.14.3 + 1.15.0 com.nimbusds diff --git a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml index b0322f92b077..4eec102625a2 100644 --- a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml @@ -35,8 +35,6 @@ https://pinot.apache.org ${basedir}/../../.. - 4.5.14 - 4.4.13 2.12.2 package @@ -46,99 +44,28 @@ org.apache.pinot pinot-spi - + software.amazon.awssdk s3 - - - org.apache.logging.log4j - log4j-to-slf4j - - - org.apache.httpcomponents - httpcore - - - org.apache.httpcomponents - httpclient - - - software.amazon.awssdk sts - - - org.apache.logging.log4j - log4j-to-slf4j - - - org.apache.httpcomponents - httpcore - - - org.apache.httpcomponents - httpclient - - software.amazon.awssdk apache-client - - org.apache.httpcomponents - httpclient - ${http.client.version} - - - org.apache.httpcomponents - httpcore - - - org.apache.logging.log4j - log4j-to-slf4j - - - - - org.apache.httpcomponents - httpcore - ${http.core.version} - - - org.reactivestreams - reactive-streams - - - io.netty - netty-codec - - - io.netty - netty-buffer - - - io.netty - netty-transport - - - io.netty - netty-common - + com.adobe.testing s3mock-testcontainers ${s3mock.version} test - - com.fasterxml.woodstox - woodstox-core - + build-shaded-jar diff --git a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml index 2d301bb2fd14..da6832e869e2 100644 --- a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml @@ -36,7 +36,7 @@ ${basedir}/../../.. 2.8.1 - 1.17.3 + 1.19.7 package diff --git a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml index bb5ed9c94de4..ab2b565f1bc0 100644 --- a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml @@ -41,5 +41,9 @@ org.apache.thrift libthrift + + org.apache.commons + commons-lang3 + diff --git a/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java b/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java index d10557d9b8b9..4be6374c85a5 100644 --- a/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java +++ b/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java @@ -97,7 +97,8 @@ protected boolean isRecord(Object value) { protected Object convertRecord(Object value) { TBase record = (TBase) value; Map convertedRecord = new HashMap<>(); - for (TFieldIdEnum tFieldIdEnum : FieldMetaData.getStructMetaDataMap(record.getClass()).keySet()) { + Set tFieldIdEnums = FieldMetaData.getStructMetaDataMap(record.getClass()).keySet(); + for (TFieldIdEnum tFieldIdEnum : tFieldIdEnums) { Object fieldValue = record.getFieldValue(tFieldIdEnum); if (fieldValue != null) { fieldValue = convert(fieldValue); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java index 9679a175dda0..c117fe551656 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java @@ -18,18 +18,14 @@ */ package org.apache.pinot.plugin.stream.kafka20; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamMetadataProvider; public class KafkaConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new KafkaPartitionLevelConsumer(clientId, _streamConfig, partition); - } - @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new KafkaStreamMetadataProvider(clientId, _streamConfig, partition); @@ -39,4 +35,11 @@ public StreamMetadataProvider createPartitionMetadataProvider(String clientId, i public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return new KafkaStreamMetadataProvider(clientId, _streamConfig); } + + @Override + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + return new KafkaPartitionLevelConsumer(clientId, _streamConfig, + partitionGroupConsumptionStatus.getPartitionGroupId()); + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java index 005b4c27b3ad..3f137b54af3c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java @@ -18,51 +18,42 @@ */ package org.apache.pinot.plugin.stream.kafka20; -import java.nio.ByteBuffer; import java.util.List; +import javax.annotation.Nullable; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -public class KafkaMessageBatch implements MessageBatch> { - private final List> _messageList; +public class KafkaMessageBatch implements MessageBatch { + private final List _messages; private final int _unfilteredMessageCount; + private final long _offsetOfNextBatch; private final long _firstOffset; - private final long _lastOffset; private final StreamMessageMetadata _lastMessageMetadata; /** + * @param messages the messages, which may be smaller than {@see unfilteredMessageCount} * @param unfilteredMessageCount how many messages were received from the topic before being filtered - * @param firstOffset the offset of the first message in the batch - * @param lastOffset the offset of the last message in the batch - * @param batch the messages, which may be smaller than {@see unfilteredMessageCount} - * @param lastMessageMetadata metadata for last filtered message in the batch, useful for estimating ingestion delay - * when a batch has all messages filtered. + * @param offsetOfNextBatch the offset of the next batch + * @param firstOffset the offset of the first unfiltered message, -1 if no unfiltered messages + * @param lastMessageMetadata metadata for the last unfiltered message in the batch, useful for estimating ingestion + * delay when a batch has all messages filtered. */ - public KafkaMessageBatch(int unfilteredMessageCount, long firstOffset, long lastOffset, - List> batch, StreamMessageMetadata lastMessageMetadata) { - _messageList = batch; - _firstOffset = firstOffset; - _lastOffset = lastOffset; + public KafkaMessageBatch(List messages, int unfilteredMessageCount, long offsetOfNextBatch, + long firstOffset, @Nullable StreamMessageMetadata lastMessageMetadata) { + _messages = messages; _unfilteredMessageCount = unfilteredMessageCount; + _offsetOfNextBatch = offsetOfNextBatch; + _firstOffset = firstOffset; _lastMessageMetadata = lastMessageMetadata; } - @Override - /** - * Returns the metadata for the last filtered message if any, null otherwise. - */ - public StreamMessageMetadata getLastMessageMetadata() { - return _lastMessageMetadata; - } - @Override public int getMessageCount() { - return _messageList.size(); + return _messages.size(); } @Override @@ -71,52 +62,24 @@ public int getUnfilteredMessageCount() { } @Override - public StreamMessage getMessageAtIndex(int index) { - return _messageList.get(index); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_messageList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageList.get(index).getValue().length; - } - - @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("This method is deprecated"); - } - - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - return new LongMsgOffset(((KafkaStreamMessage) _messageList.get(index)).getNextOffset()); + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); } @Override public StreamPartitionMsgOffset getOffsetOfNextBatch() { - return new LongMsgOffset(_lastOffset + 1); - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return _messageList.get(index).getMetadata(); + return new LongMsgOffset(_offsetOfNextBatch); } + @Nullable @Override - public byte[] getMessageBytesAtIndex(int index) { - return _messageList.get(index).getValue(); - } - - @Override - public StreamMessage getStreamMessage(int index) { - return _messageList.get(index); + public StreamPartitionMsgOffset getFirstMessageOffset() { + return _firstOffset >= 0 ? new LongMsgOffset(_firstOffset) : null; } + @Nullable @Override - public StreamPartitionMsgOffset getFirstMessageOffset() { - return new LongMsgOffset(_firstOffset); + public StreamMessageMetadata getLastMessageMetadata() { + return _lastMessageMetadata; } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java deleted file mode 100644 index 86708ae715ae..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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.pinot.plugin.stream.kafka20; - -import java.util.HashMap; -import java.util.Map; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.Headers; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.RowMetadata; - - -@FunctionalInterface -public interface KafkaMetadataExtractor { - static KafkaMetadataExtractor build(boolean populateMetadata) { - return record -> { - long recordTimestamp = record.timestamp(); - Map metadataMap = new HashMap<>(); - metadataMap.put(KafkaStreamMessageMetadata.METADATA_OFFSET_KEY, String.valueOf(record.offset())); - metadataMap.put(KafkaStreamMessageMetadata.RECORD_TIMESTAMP_KEY, String.valueOf(recordTimestamp)); - - if (!populateMetadata) { - return new KafkaStreamMessageMetadata(recordTimestamp, RowMetadata.EMPTY_ROW, metadataMap); - } - GenericRow headerGenericRow = new GenericRow(); - Headers headers = record.headers(); - if (headers != null) { - Header[] headersArray = headers.toArray(); - for (Header header : headersArray) { - headerGenericRow.putValue(header.key(), header.value()); - } - } - return new KafkaStreamMessageMetadata(record.timestamp(), headerGenericRow, metadataMap); - }; - } - - RowMetadata extract(ConsumerRecord consumerRecord); -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java index f0512cc8b339..15b6c08496fa 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java @@ -51,7 +51,6 @@ public abstract class KafkaPartitionLevelConnectionHandler { protected final String _topic; protected final Consumer _consumer; protected final TopicPartition _topicPartition; - protected final KafkaMetadataExtractor _kafkaMetadataExtractor; public KafkaPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig, int partition) { _config = new KafkaPartitionLevelStreamConfig(streamConfig); @@ -71,7 +70,6 @@ public KafkaPartitionLevelConnectionHandler(String clientId, StreamConfig stream _consumer = createConsumer(consumerProp); _topicPartition = new TopicPartition(_topic, _partition); _consumer.assign(Collections.singletonList(_topicPartition)); - _kafkaMetadataExtractor = KafkaMetadataExtractor.build(_config.isPopulateMetadata()); } private Consumer createConsumer(Properties consumerProp) { diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java index df51d2fda9a3..36a74c1e6530 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java @@ -22,14 +22,17 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.utils.Bytes; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; @@ -37,7 +40,7 @@ public class KafkaPartitionLevelConsumer extends KafkaPartitionLevelConnectionHandler - implements PartitionLevelConsumer { + implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaPartitionLevelConsumer.class); private long _lastFetchedOffset = -1; @@ -47,18 +50,10 @@ public KafkaPartitionLevelConsumer(String clientId, StreamConfig streamConfig, i } @Override - public MessageBatch> fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { - final long startOffset = ((LongMsgOffset) startMsgOffset).getOffset(); - final long endOffset = endMsgOffset == null ? Long.MAX_VALUE : ((LongMsgOffset) endMsgOffset).getOffset(); - return fetchMessages(startOffset, endOffset, timeoutMillis); - } - - public synchronized MessageBatch> fetchMessages(long startOffset, long endOffset, - int timeoutMillis) { + public synchronized KafkaMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { + long startOffset = ((LongMsgOffset) startMsgOffset).getOffset(); if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Polling partition: {}, startOffset: {}, endOffset: {} timeout: {}ms", _topicPartition, startOffset, - endOffset, timeoutMillis); + LOGGER.debug("Polling partition: {}, startOffset: {}, timeout: {}ms", _topicPartition, startOffset, timeoutMs); } if (_lastFetchedOffset < 0 || _lastFetchedOffset != startOffset - 1) { if (LOGGER.isDebugEnabled()) { @@ -66,34 +61,49 @@ public synchronized MessageBatch> fetchMessages(long start } _consumer.seek(_topicPartition, startOffset); } - ConsumerRecords consumerRecords = _consumer.poll(Duration.ofMillis(timeoutMillis)); - List> messageAndOffsets = consumerRecords.records(_topicPartition); - List> filtered = new ArrayList<>(messageAndOffsets.size()); - long firstOffset = startOffset; - long lastOffset = startOffset; - StreamMessageMetadata rowMetadata = null; - if (!messageAndOffsets.isEmpty()) { - firstOffset = messageAndOffsets.get(0).offset(); - } - for (ConsumerRecord messageAndOffset : messageAndOffsets) { - long offset = messageAndOffset.offset(); - _lastFetchedOffset = offset; - if (offset >= startOffset && (endOffset > offset || endOffset < 0)) { - Bytes message = messageAndOffset.value(); - rowMetadata = (StreamMessageMetadata) _kafkaMetadataExtractor.extract(messageAndOffset); + ConsumerRecords consumerRecords = _consumer.poll(Duration.ofMillis(timeoutMs)); + List> records = consumerRecords.records(_topicPartition); + List filteredRecords = new ArrayList<>(records.size()); + long firstOffset = -1; + long offsetOfNextBatch = startOffset; + StreamMessageMetadata lastMessageMetadata = null; + if (!records.isEmpty()) { + firstOffset = records.get(0).offset(); + _lastFetchedOffset = records.get(records.size() - 1).offset(); + offsetOfNextBatch = _lastFetchedOffset + 1; + for (ConsumerRecord record : records) { + StreamMessageMetadata messageMetadata = extractMessageMetadata(record); + Bytes message = record.value(); if (message != null) { - String key = messageAndOffset.key(); + String key = record.key(); byte[] keyBytes = key != null ? key.getBytes(StandardCharsets.UTF_8) : null; - filtered.add(new KafkaStreamMessage(keyBytes, message.get(), rowMetadata)); + filteredRecords.add(new BytesStreamMessage(keyBytes, message.get(), messageMetadata)); } else if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Tombstone message at offset: {}", offset); + LOGGER.debug("Tombstone message at offset: {}", record.offset()); + } + lastMessageMetadata = messageMetadata; + } + } + return new KafkaMessageBatch(filteredRecords, records.size(), offsetOfNextBatch, firstOffset, lastMessageMetadata); + } + + private StreamMessageMetadata extractMessageMetadata(ConsumerRecord record) { + long timestamp = record.timestamp(); + long offset = record.offset(); + StreamMessageMetadata.Builder builder = new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(timestamp) + .setOffset(new LongMsgOffset(offset), new LongMsgOffset(offset + 1)); + if (_config.isPopulateMetadata()) { + Headers headers = record.headers(); + if (headers != null) { + GenericRow headerGenericRow = new GenericRow(); + for (Header header : headers.toArray()) { + headerGenericRow.putValue(header.key(), header.value()); } - lastOffset = offset; - } else if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Ignoring message at offset: {} (outside of offset range [{}, {}))", offset, startOffset, - endOffset); + builder.setHeaders(headerGenericRow); } + builder.setMetadata(Map.of(KafkaStreamMessageMetadata.RECORD_TIMESTAMP_KEY, String.valueOf(timestamp), + KafkaStreamMessageMetadata.METADATA_OFFSET_KEY, String.valueOf(offset))); } - return new KafkaMessageBatch(messageAndOffsets.size(), firstOffset, lastOffset, filtered, rowMetadata); + return builder.build(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java index ada4e22bbb52..7cecab28ef03 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java @@ -23,11 +23,12 @@ import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.stream.StreamMessageMetadata; - +// TODO: Make it a util class public class KafkaStreamMessageMetadata extends StreamMessageMetadata { public static final String METADATA_OFFSET_KEY = "offset"; public static final String RECORD_TIMESTAMP_KEY = "recordTimestamp"; + @Deprecated public KafkaStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { super(recordIngestionTimeMs, headers, metadata); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java index e783e091ae90..6719a722c761 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java @@ -32,17 +32,22 @@ import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.RowMetadata; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; import org.apache.pinot.spi.stream.StreamMessage; -import org.testng.Assert; +import org.apache.pinot.spi.stream.StreamMessageMetadata; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + /** * Tests for the KafkaPartitionLevelConsumer. @@ -104,8 +109,7 @@ public void tearDown() } @Test - public void testBuildConsumer() - throws Exception { + public void testBuildConsumer() { String streamType = "kafka"; String streamKafkaTopicName = "theTopic"; String streamKafkaBrokerList = _kafkaBrokerAddress; @@ -126,27 +130,25 @@ public void testBuildConsumer() // test default value KafkaPartitionLevelConsumer kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, 0); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); - Assert.assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_BUFFER_SIZE_DEFAULT, + assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_BUFFER_SIZE_DEFAULT, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); - Assert.assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_SOCKET_TIMEOUT_DEFAULT, + assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_SOCKET_TIMEOUT_DEFAULT, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); // test parsing values - Assert.assertEquals(10000, - kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherSizeBytes()); - Assert.assertEquals(20000, - kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherMinBytes()); + assertEquals(10000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherSizeBytes()); + assertEquals(20000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherMinBytes()); // test user defined values streamConfigMap.put("stream.kafka.buffer.size", "100"); streamConfigMap.put("stream.kafka.socket.timeout", "1000"); streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, 0); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); - Assert.assertEquals(100, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); - Assert.assertEquals(1000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); + assertEquals(100, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); + assertEquals(1000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); } @Test @@ -167,7 +169,7 @@ public void testGetPartitionCount() { StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); KafkaStreamMetadataProvider streamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig); - Assert.assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 1); + assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 1); streamConfigMap = new HashMap<>(); streamConfigMap.put("streamType", streamType); @@ -179,12 +181,11 @@ public void testGetPartitionCount() { streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); streamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig); - Assert.assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 2); + assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 2); } @Test - public void testFetchMessages() - throws Exception { + public void testFetchMessages() { String streamType = "kafka"; String streamKafkaTopicName = "theTopic"; String streamKafkaBrokerList = _kafkaBrokerAddress; @@ -204,18 +205,16 @@ public void testFetchMessages() int partition = 0; KafkaPartitionLevelConsumer kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, partition); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); } @Test - public void testFetchOffsets() - throws Exception { + public void testFetchOffsets() { testFetchOffsets(TEST_TOPIC_1); testFetchOffsets(TEST_TOPIC_2); } - private void testFetchOffsets(String topic) - throws Exception { + private void testFetchOffsets(String topic) { String streamType = "kafka"; String streamKafkaBrokerList = _kafkaBrokerAddress; String streamKafkaConsumerType = "simple"; @@ -235,14 +234,14 @@ private void testFetchOffsets(String topic) for (int partition = 0; partition < numPartitions; partition++) { KafkaStreamMetadataProvider kafkaStreamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig, partition); - Assert.assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetSmallest(), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetAsPeriod("2d"), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION) - .compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset(new OffsetCriteria.OffsetCriteriaBuilder() - .withOffsetAsTimestamp(Instant.now().toString()), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( + assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( + kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + new OffsetCriteria.OffsetCriteriaBuilder().withOffsetAsTimestamp(Instant.now().toString()), 10000)), 0); + assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetLargest(), 10000)), 0); } @@ -272,102 +271,100 @@ private void testConsumer(String topic) streamConfigMap.put("stream.kafka.decoder.class.name", "decoderClass"); StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - final StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); int numPartitions = new KafkaStreamMetadataProvider(clientId, streamConfig).fetchPartitionCount(10000); for (int partition = 0; partition < numPartitions; partition++) { - final PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); + PartitionGroupConsumer consumer = streamConsumerFactory.createPartitionGroupConsumer(clientId, + new PartitionGroupConsumptionStatus(partition, 0, new LongMsgOffset(0), + new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), "CONSUMING")); // Test consume a large batch, only 500 records will be returned. - final MessageBatch batch1 = - consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch1.getMessageCount(), 500); - for (int i = 0; i < batch1.getMessageCount(); i++) { - StreamMessage streamMessage = batch1.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + MessageBatch messageBatch = consumer.fetchMessages(new LongMsgOffset(0), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + i); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), i + 1); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "500"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "0"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "499"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "500"); + // Test second half batch - final MessageBatch batch2 = - consumer.fetchMessages(new LongMsgOffset(500), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch2.getMessageCount(), 500); - for (int i = 0; i < batch2.getMessageCount(); i++) { - StreamMessage streamMessage = batch2.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(500), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (500 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 500 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 500 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 501 + i); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "1000"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "500"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "999"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "1000"); + // Some random range - final MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(35), 10000); - Assert.assertEquals(batch3.getMessageCount(), 25); - for (int i = 0; i < batch3.getMessageCount(); i++) { - StreamMessage streamMessage = batch3.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(10), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (10 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 10 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 10 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 11 + i); } - } - } - - @Test - public void testMessageMetadata() - throws Exception { - testMessageMetadata(TEST_TOPIC_1); - testMessageMetadata(TEST_TOPIC_2); - } - - private void testMessageMetadata(String topic) - throws TimeoutException { - String streamType = "kafka"; - String streamKafkaBrokerList = _kafkaBrokerAddress; - String streamKafkaConsumerType = "simple"; - String clientId = "clientId"; - String tableNameWithType = "tableName_REALTIME"; + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "510"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "10"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "509"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "510"); - Map streamConfigMap = new HashMap<>(); - streamConfigMap.put("streamType", streamType); - streamConfigMap.put("stream.kafka.topic.name", topic); - streamConfigMap.put("stream.kafka.broker.list", streamKafkaBrokerList); - streamConfigMap.put("stream.kafka.consumer.type", streamKafkaConsumerType); - streamConfigMap.put("stream.kafka.consumer.factory.class.name", getKafkaConsumerFactoryName()); - streamConfigMap.put("stream.kafka.decoder.class.name", "decoderClass"); - streamConfigMap.put("stream.kafka.metadata.populate", "true"); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - - final StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - int numPartitions = new KafkaStreamMetadataProvider(clientId, streamConfig).fetchPartitionCount(10000); - for (int partition = 0; partition < numPartitions; partition++) { - final PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); - - // Test consume a large batch, only 500 records will be returned. - final MessageBatch batch1 = - consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch1.getMessageCount(), 500); - for (int i = 0; i < batch1.getMessageCount(); i++) { - final RowMetadata metadata = batch1.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + i); - } - // Test second half batch - final MessageBatch batch2 = - consumer.fetchMessages(new LongMsgOffset(500), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch2.getMessageCount(), 500); - for (int i = 0; i < batch2.getMessageCount(); i++) { - final RowMetadata metadata = batch2.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + (500 + i)); - } // Some random range - final MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(35), 10000); - Assert.assertEquals(batch3.getMessageCount(), 25); - for (int i = 0; i < batch3.getMessageCount(); i++) { - final RowMetadata metadata = batch3.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + (10 + i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(610), 10000); + assertEquals(messageBatch.getMessageCount(), 390); + assertEquals(messageBatch.getUnfilteredMessageCount(), 390); + for (int i = 0; i < 390; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (610 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 610 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 610 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 611 + i); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "1000"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "610"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "999"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "1000"); } } @@ -389,35 +386,17 @@ public void testOffsetsExpired() StreamConfig streamConfig = new StreamConfig("tableName_REALTIME", streamConfigMap); StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer("clientId", 0); + PartitionGroupConsumer consumer = streamConsumerFactory.createPartitionGroupConsumer("clientId", + new PartitionGroupConsumptionStatus(0, 0, new LongMsgOffset(0), + new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), "CONSUMING")); // Start offset has expired. Automatically reset to earliest available and fetch whatever available - MessageBatch batch1 = consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(400), 10000); - Assert.assertEquals(batch1.getMessageCount(), 200); - for (int i = 0; i < batch1.getMessageCount(); i++) { - byte[] msg = (byte[]) batch1.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 200)); - } - Assert.assertEquals(batch1.getOffsetOfNextBatch().toString(), "400"); - - // Start and end offset has expired - MessageBatch batch2 = consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(100), 10000); - Assert.assertEquals(batch2.getMessageCount(), 0); - - MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(201), new LongMsgOffset(401), 10000); - Assert.assertEquals(batch3.getMessageCount(), 200); - for (int i = 0; i < batch3.getMessageCount(); i++) { - byte[] msg = (byte[]) batch3.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 201)); - } - Assert.assertEquals(batch3.getOffsetOfNextBatch().toString(), "401"); - - MessageBatch batch4 = consumer.fetchMessages(new LongMsgOffset(0), null, 10000); - Assert.assertEquals(batch4.getMessageCount(), 500); - for (int i = 0; i < batch4.getMessageCount(); i++) { - byte[] msg = (byte[]) batch4.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 200)); + MessageBatch messageBatch = consumer.fetchMessages(new LongMsgOffset(0), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + assertEquals(new String((byte[]) messageBatch.getStreamMessage(i).getValue()), "sample_msg_" + (200 + i)); } - Assert.assertEquals(batch4.getOffsetOfNextBatch().toString(), "700"); + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "700"); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml index 18e653a6f7eb..919bd1d19e29 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml @@ -36,9 +36,8 @@ ${basedir}/../../.. package - 5.2.0 1.0.2 - 0.2.19 + 0.2.23 @@ -122,9 +121,8 @@ - org.easymock - easymock - ${easymock.version} + org.mockito + mockito-core test diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java index 5c021e4b9396..7ae08cdbb5b2 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java @@ -19,6 +19,7 @@ package org.apache.pinot.plugin.stream.kinesis; import com.google.common.annotations.VisibleForTesting; +import java.io.Closeable; import java.net.URI; import java.net.URISyntaxException; import java.util.List; @@ -42,115 +43,80 @@ /** * Manages the Kinesis stream connection, given the stream name and aws region */ -public class KinesisConnectionHandler { - protected KinesisClient _kinesisClient; - private final String _stream; - private final String _region; - private final String _accessKey; - private final String _secretKey; - private final String _endpoint; - private final KinesisConfig _kinesisConfig; - protected final KinesisMetadataExtractor _kinesisMetadataExtractor; +public class KinesisConnectionHandler implements Closeable { + protected final KinesisConfig _config; + protected final KinesisClient _kinesisClient; - public KinesisConnectionHandler(KinesisConfig kinesisConfig) { - _stream = kinesisConfig.getStreamTopicName(); - _region = kinesisConfig.getAwsRegion(); - _accessKey = kinesisConfig.getAccessKey(); - _secretKey = kinesisConfig.getSecretKey(); - _endpoint = kinesisConfig.getEndpoint(); - _kinesisConfig = kinesisConfig; - _kinesisMetadataExtractor = KinesisMetadataExtractor.build(kinesisConfig.isPopulateMetadata()); - createConnection(); + public KinesisConnectionHandler(KinesisConfig config) { + _config = config; + _kinesisClient = createClient(); } @VisibleForTesting - public KinesisConnectionHandler(KinesisConfig kinesisConfig, KinesisClient kinesisClient) { - _stream = kinesisConfig.getStreamTopicName(); - _region = kinesisConfig.getAwsRegion(); - _accessKey = kinesisConfig.getAccessKey(); - _secretKey = kinesisConfig.getSecretKey(); - _endpoint = kinesisConfig.getEndpoint(); - _kinesisConfig = kinesisConfig; - _kinesisMetadataExtractor = KinesisMetadataExtractor.build(kinesisConfig.isPopulateMetadata()); + public KinesisConnectionHandler(KinesisConfig config, KinesisClient kinesisClient) { + _config = config; _kinesisClient = kinesisClient; } - /** - * Lists all shards of the stream - */ - public List getShards() { - ListShardsResponse listShardsResponse = - _kinesisClient.listShards(ListShardsRequest.builder().streamName(_stream).build()); - return listShardsResponse.shards(); - } - - /** - * Creates a Kinesis client for the stream - */ - public void createConnection() { - if (_kinesisClient == null) { - KinesisClientBuilder kinesisClientBuilder; + private KinesisClient createClient() { + KinesisClientBuilder kinesisClientBuilder; + + AwsCredentialsProvider awsCredentialsProvider; + String accessKey = _config.getAccessKey(); + String secretKey = _config.getSecretKey(); + if (StringUtils.isNotBlank(accessKey) && StringUtils.isNotBlank(secretKey)) { + AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); + awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); + } else { + awsCredentialsProvider = DefaultCredentialsProvider.builder().build(); + } - AwsCredentialsProvider awsCredentialsProvider; - if (StringUtils.isNotBlank(_accessKey) && StringUtils.isNotBlank(_secretKey)) { - AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(_accessKey, _secretKey); - awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); + if (_config.isIamRoleBasedAccess()) { + AssumeRoleRequest.Builder assumeRoleRequestBuilder = + AssumeRoleRequest.builder().roleArn(_config.getRoleArn()).roleSessionName(_config.getRoleSessionName()) + .durationSeconds(_config.getSessionDurationSeconds()); + AssumeRoleRequest assumeRoleRequest; + String externalId = _config.getExternalId(); + if (StringUtils.isNotBlank(externalId)) { + assumeRoleRequest = assumeRoleRequestBuilder.externalId(externalId).build(); } else { - awsCredentialsProvider = DefaultCredentialsProvider.builder().build(); + assumeRoleRequest = assumeRoleRequestBuilder.build(); } - - if (_kinesisConfig.isIamRoleBasedAccess()) { - AssumeRoleRequest.Builder assumeRoleRequestBuilder = - AssumeRoleRequest.builder() - .roleArn(_kinesisConfig.getRoleArn()) - .roleSessionName(_kinesisConfig.getRoleSessionName()) - .durationSeconds(_kinesisConfig.getSessionDurationSeconds()); - - AssumeRoleRequest assumeRoleRequest; - if (StringUtils.isNotEmpty(_kinesisConfig.getExternalId())) { - assumeRoleRequest = assumeRoleRequestBuilder - .externalId(_kinesisConfig.getExternalId()) + StsClient stsClient = + StsClient.builder().region(Region.of(_config.getAwsRegion())).credentialsProvider(awsCredentialsProvider) .build(); - } else { - assumeRoleRequest = assumeRoleRequestBuilder.build(); - } + awsCredentialsProvider = + StsAssumeRoleCredentialsProvider.builder().stsClient(stsClient).refreshRequest(assumeRoleRequest) + .asyncCredentialUpdateEnabled(_config.isAsyncSessionUpdateEnabled()).build(); + } - StsClient stsClient = - StsClient.builder() - .region(Region.of(_region)) - .credentialsProvider(awsCredentialsProvider) - .build(); + kinesisClientBuilder = + KinesisClient.builder().region(Region.of(_config.getAwsRegion())).credentialsProvider(awsCredentialsProvider) + .httpClientBuilder(new ApacheSdkHttpService().createHttpClientBuilder()); - awsCredentialsProvider = - StsAssumeRoleCredentialsProvider.builder() - .stsClient(stsClient) - .refreshRequest(assumeRoleRequest) - .asyncCredentialUpdateEnabled(_kinesisConfig.isAsyncSessionUpdateEnabled()) - .build(); + String endpoint = _config.getEndpoint(); + if (StringUtils.isNotBlank(endpoint)) { + try { + kinesisClientBuilder = kinesisClientBuilder.endpointOverride(new URI(endpoint)); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("URI syntax is not correctly specified for endpoint: " + endpoint, e); } + } - kinesisClientBuilder = - KinesisClient.builder() - .region(Region.of(_region)) - .credentialsProvider(awsCredentialsProvider) - .httpClientBuilder(new ApacheSdkHttpService().createHttpClientBuilder()); - - if (StringUtils.isNotBlank(_endpoint)) { - try { - kinesisClientBuilder = kinesisClientBuilder.endpointOverride(new URI(_endpoint)); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("URI syntax is not correctly specified for endpoint: " + _endpoint, e); - } - } + return kinesisClientBuilder.build(); + } - _kinesisClient = kinesisClientBuilder.build(); - } + /** + * Lists all shards of the stream + */ + public List getShards() { + ListShardsResponse listShardsResponse = + _kinesisClient.listShards(ListShardsRequest.builder().streamName(_config.getStreamTopicName()).build()); + return listShardsResponse.shards(); } + @Override public void close() { - if (_kinesisClient != null) { - _kinesisClient.close(); - _kinesisClient = null; - } + _kinesisClient.close(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java index 53d0fdde2c88..3d307bfbb25e 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java @@ -19,10 +19,8 @@ package org.apache.pinot.plugin.stream.kinesis; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -30,7 +28,9 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +43,7 @@ import software.amazon.awssdk.services.kinesis.model.InvalidArgumentException; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; @@ -52,106 +53,58 @@ */ public class KinesisConsumer extends KinesisConnectionHandler implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(KinesisConsumer.class); - public static final long SLEEP_TIME_BETWEEN_REQUESTS = 1000L; - private final String _streamTopicName; - private final int _numMaxRecordsToFetch; - private final ExecutorService _executorService; - private final ShardIteratorType _shardIteratorType; - private final int _rpsLimit; + private static final long SLEEP_TIME_BETWEEN_REQUESTS = 1000L; - public KinesisConsumer(KinesisConfig kinesisConfig) { - super(kinesisConfig); - _streamTopicName = kinesisConfig.getStreamTopicName(); - _numMaxRecordsToFetch = kinesisConfig.getNumMaxRecordsToFetch(); - _shardIteratorType = kinesisConfig.getShardIteratorType(); - _rpsLimit = kinesisConfig.getRpsLimit(); - _executorService = Executors.newSingleThreadExecutor(); + // TODO: Revisit the logic of using a separate executor to manage the request timeout. Currently it is not thread safe + private final ExecutorService _executorService = Executors.newSingleThreadExecutor(); + + public KinesisConsumer(KinesisConfig config) { + super(config); + LOGGER.info("Created Kinesis consumer with topic: {}, RPS limit: {}, max records per fetch: {}", + config.getStreamTopicName(), config.getRpsLimit(), config.getNumMaxRecordsToFetch()); } @VisibleForTesting - public KinesisConsumer(KinesisConfig kinesisConfig, KinesisClient kinesisClient) { - super(kinesisConfig, kinesisClient); - _kinesisClient = kinesisClient; - _streamTopicName = kinesisConfig.getStreamTopicName(); - _numMaxRecordsToFetch = kinesisConfig.getNumMaxRecordsToFetch(); - _shardIteratorType = kinesisConfig.getShardIteratorType(); - _rpsLimit = kinesisConfig.getRpsLimit(); - _executorService = Executors.newSingleThreadExecutor(); + public KinesisConsumer(KinesisConfig config, KinesisClient kinesisClient) { + super(config, kinesisClient); } /** * Fetch records from the Kinesis stream between the start and end KinesisCheckpoint */ @Override - public KinesisRecordsBatch fetchMessages(StreamPartitionMsgOffset startCheckpoint, - StreamPartitionMsgOffset endCheckpoint, int timeoutMs) { - List recordList = new ArrayList<>(); - Future kinesisFetchResultFuture = - _executorService.submit(() -> getResult(startCheckpoint, endCheckpoint, recordList)); - + public KinesisMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { + KinesisPartitionGroupOffset startOffset = (KinesisPartitionGroupOffset) startMsgOffset; + List messages = new ArrayList<>(); + Future kinesisFetchResultFuture = + _executorService.submit(() -> getResult(startOffset, messages)); try { return kinesisFetchResultFuture.get(timeoutMs, TimeUnit.MILLISECONDS); } catch (TimeoutException e) { kinesisFetchResultFuture.cancel(true); - return handleException((KinesisPartitionGroupOffset) startCheckpoint, recordList); } catch (Exception e) { - return handleException((KinesisPartitionGroupOffset) startCheckpoint, recordList); + // Ignored } + return buildKinesisMessageBatch(startOffset, messages, false); } - private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, - List recordList) { - KinesisPartitionGroupOffset kinesisStartCheckpoint = (KinesisPartitionGroupOffset) startOffset; - + private KinesisMessageBatch getResult(KinesisPartitionGroupOffset startOffset, List messages) { try { - if (_kinesisClient == null) { - createConnection(); - } - - // TODO: iterate upon all the shardIds in the map - // Okay for now, since we have assumed that every partition group contains a single shard - Map startShardToSequenceMap = kinesisStartCheckpoint.getShardToStartSequenceMap(); - Preconditions.checkState(startShardToSequenceMap.size() == 1, - "Only 1 shard per consumer supported. Found: %s, in startShardToSequenceMap", - startShardToSequenceMap.keySet()); - Map.Entry startShardToSequenceNum = startShardToSequenceMap.entrySet().iterator().next(); - String shardIterator = getShardIterator(startShardToSequenceNum.getKey(), startShardToSequenceNum.getValue()); - - String kinesisEndSequenceNumber = null; - - if (endOffset != null) { - KinesisPartitionGroupOffset kinesisEndCheckpoint = (KinesisPartitionGroupOffset) endOffset; - Map endShardToSequenceMap = kinesisEndCheckpoint.getShardToStartSequenceMap(); - Preconditions.checkState(endShardToSequenceMap.size() == 1, - "Only 1 shard per consumer supported. Found: %s, in endShardToSequenceMap", endShardToSequenceMap.keySet()); - kinesisEndSequenceNumber = endShardToSequenceMap.values().iterator().next(); - } - - String nextStartSequenceNumber; - boolean isEndOfShard = false; + String shardId = startOffset.getShardId(); + String shardIterator = getShardIterator(shardId, startOffset.getSequenceNumber()); + boolean endOfShard = false; long currentWindow = System.currentTimeMillis() / SLEEP_TIME_BETWEEN_REQUESTS; int currentWindowRequests = 0; while (shardIterator != null) { GetRecordsRequest getRecordsRequest = GetRecordsRequest.builder().shardIterator(shardIterator).build(); - long requestSentTime = System.currentTimeMillis() / 1000; GetRecordsResponse getRecordsResponse = _kinesisClient.getRecords(getRecordsRequest); - - if (!getRecordsResponse.records().isEmpty()) { - getRecordsResponse.records().forEach(record -> { - recordList.add( - new KinesisStreamMessage(record.partitionKey().getBytes(StandardCharsets.UTF_8), - record.data().asByteArray(), record.sequenceNumber(), - (KinesisStreamMessageMetadata) _kinesisMetadataExtractor.extract(record), - record.data().asByteArray().length)); - }); - nextStartSequenceNumber = recordList.get(recordList.size() - 1).sequenceNumber(); - - if (kinesisEndSequenceNumber != null && kinesisEndSequenceNumber.compareTo(nextStartSequenceNumber) <= 0) { - break; + List records = getRecordsResponse.records(); + if (!records.isEmpty()) { + for (Record record : records) { + messages.add(extractStreamMessage(record, shardId)); } - - if (recordList.size() >= _numMaxRecordsToFetch) { + if (messages.size() >= _config.getNumMaxRecordsToFetch()) { break; } } @@ -159,7 +112,7 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre if (getRecordsResponse.hasChildShards() && !getRecordsResponse.childShards().isEmpty()) { //This statement returns true only when end of current shard has reached. // hasChildShards only checks if the childShard is null and is a valid instance. - isEndOfShard = true; + endOfShard = true; break; } @@ -178,7 +131,7 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre currentWindowRequests = 0; } - if (currentWindowRequests >= _rpsLimit) { + if (currentWindowRequests >= _config.getNumMaxRecordsToFetch()) { try { Thread.sleep(SLEEP_TIME_BETWEEN_REQUESTS); } catch (InterruptedException e) { @@ -188,20 +141,16 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre } } - return new KinesisRecordsBatch(recordList, startShardToSequenceNum.getKey(), isEndOfShard); + return buildKinesisMessageBatch(startOffset, messages, endOfShard); } catch (IllegalStateException e) { debugOrLogWarning("Illegal state exception, connection is broken", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ProvisionedThroughputExceededException e) { debugOrLogWarning("The request rate for the stream is too high", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ExpiredIteratorException e) { debugOrLogWarning("ShardIterator expired while trying to fetch records", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ResourceNotFoundException | InvalidArgumentException e) { // aws errors LOGGER.error("Encountered AWS error while attempting to fetch records", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (KinesisException e) { debugOrLogWarning("Encountered unknown unrecoverable AWS exception", e); throw new RuntimeException(e); @@ -209,12 +158,12 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre if (!(e.getCause() instanceof InterruptedException)) { debugOrLogWarning("Task aborted due to exception", e); } - return handleException(kinesisStartCheckpoint, recordList); } catch (Throwable e) { // non transient errors LOGGER.error("Unknown fetchRecords exception", e); throw new RuntimeException(e); } + return buildKinesisMessageBatch(startOffset, messages, false); } private void debugOrLogWarning(String message, Throwable throwable) { @@ -225,32 +174,48 @@ private void debugOrLogWarning(String message, Throwable throwable) { } } - private KinesisRecordsBatch handleException(KinesisPartitionGroupOffset start, - List recordList) { - String shardId = start.getShardToStartSequenceMap().entrySet().iterator().next().getKey(); - - if (!recordList.isEmpty()) { - String nextStartSequenceNumber = recordList.get(recordList.size() - 1).sequenceNumber(); - Map newCheckpoint = new HashMap<>(start.getShardToStartSequenceMap()); - newCheckpoint.put(newCheckpoint.keySet().iterator().next(), nextStartSequenceNumber); + private KinesisMessageBatch buildKinesisMessageBatch(KinesisPartitionGroupOffset startOffset, + List messages, boolean endOfShard) { + KinesisPartitionGroupOffset offsetOfNextBatch; + if (messages.isEmpty()) { + offsetOfNextBatch = startOffset; + } else { + StreamMessageMetadata lastMessageMetadata = messages.get(messages.size() - 1).getMetadata(); + assert lastMessageMetadata != null; + offsetOfNextBatch = (KinesisPartitionGroupOffset) lastMessageMetadata.getNextOffset(); } - return new KinesisRecordsBatch(recordList, shardId, false); + return new KinesisMessageBatch(messages, offsetOfNextBatch, endOfShard); } private String getShardIterator(String shardId, String sequenceNumber) { GetShardIteratorRequest.Builder requestBuilder = - GetShardIteratorRequest.builder().streamName(_streamTopicName).shardId(shardId); - + GetShardIteratorRequest.builder().streamName(_config.getStreamTopicName()).shardId(shardId); if (sequenceNumber != null) { requestBuilder = requestBuilder.startingSequenceNumber(sequenceNumber) .shardIteratorType(ShardIteratorType.AFTER_SEQUENCE_NUMBER); } else { - requestBuilder = requestBuilder.shardIteratorType(_shardIteratorType); + requestBuilder = requestBuilder.shardIteratorType(_config.getShardIteratorType()); } - return _kinesisClient.getShardIterator(requestBuilder.build()).shardIterator(); } + private BytesStreamMessage extractStreamMessage(Record record, String shardId) { + byte[] key = record.partitionKey().getBytes(StandardCharsets.UTF_8); + byte[] value = record.data().asByteArray(); + long timestamp = record.approximateArrivalTimestamp().toEpochMilli(); + String sequenceNumber = record.sequenceNumber(); + KinesisPartitionGroupOffset offset = new KinesisPartitionGroupOffset(shardId, sequenceNumber); + // NOTE: Use the same offset as next offset because the consumer starts consuming AFTER the start sequence number. + StreamMessageMetadata.Builder builder = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(timestamp).setOffset(offset, offset); + if (_config.isPopulateMetadata()) { + builder.setMetadata(Map.of(KinesisStreamMessageMetadata.APPRX_ARRIVAL_TIMESTAMP_KEY, String.valueOf(timestamp), + KinesisStreamMessageMetadata.SEQUENCE_NUMBER_KEY, sequenceNumber)); + } + StreamMessageMetadata metadata = builder.build(); + return new BytesStreamMessage(key, value, metadata); + } + @Override public void close() { super.close(); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java index 6f81632c8c64..bd7d9ad8c09f 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java @@ -20,7 +20,6 @@ import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -31,10 +30,6 @@ */ public class KinesisConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - throw new UnsupportedOperationException(); - } @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new KinesisStreamMetadataProvider(clientId, _streamConfig); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java new file mode 100644 index 000000000000..866e99594fe4 --- /dev/null +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java @@ -0,0 +1,61 @@ +/** + * 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.pinot.plugin.stream.kinesis; + +import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; + + +/** + * A {@link MessageBatch} for collecting records from the Kinesis stream + */ +public class KinesisMessageBatch implements MessageBatch { + private final List _messages; + private final KinesisPartitionGroupOffset _offsetOfNextBatch; + private final boolean _endOfShard; + + public KinesisMessageBatch(List messages, KinesisPartitionGroupOffset offsetOfNextBatch, + boolean endOfShard) { + _messages = messages; + _offsetOfNextBatch = offsetOfNextBatch; + _endOfShard = endOfShard; + } + + @Override + public int getMessageCount() { + return _messages.size(); + } + + @Override + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); + } + + @Override + public StreamPartitionMsgOffset getOffsetOfNextBatch() { + return _offsetOfNextBatch; + } + + @Override + public boolean isEndOfPartitionGroup() { + return _endOfShard; + } +} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java deleted file mode 100644 index 963ff8f689ec..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * 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.pinot.plugin.stream.kinesis; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.pinot.spi.stream.RowMetadata; -import software.amazon.awssdk.services.kinesis.model.Record; - -// TODO: Make this an interface/api in the stream spis st StreamMessageMetadata extract(T streamTypeRecord) - -public interface KinesisMetadataExtractor { - static KinesisMetadataExtractor build(boolean populateMetadata) { - return record -> { - long recordTimestamp = record.approximateArrivalTimestamp().toEpochMilli(); - - if (!populateMetadata) { - return new KinesisStreamMessageMetadata(recordTimestamp, null, Collections.emptyMap()); - } - String seqNumber = record.sequenceNumber(); - Map metadataMap = new HashMap<>(); - metadataMap.put(KinesisStreamMessageMetadata.APPRX_ARRIVAL_TIMESTAMP_KEY, String.valueOf(recordTimestamp)); - metadataMap.put(KinesisStreamMessageMetadata.SEQUENCE_NUMBER_KEY, seqNumber); - return new KinesisStreamMessageMetadata(recordTimestamp, null, metadataMap); - }; - } - RowMetadata extract(Record record); -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java index f745b114ab46..5f17badfe622 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.plugin.stream.kinesis; -import java.io.IOException; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -35,16 +34,6 @@ public void init(StreamConfig streamConfig) { @Override public StreamPartitionMsgOffset create(String offsetStr) { - try { - return new KinesisPartitionGroupOffset(offsetStr); - } catch (IOException e) { - throw new IllegalStateException( - "Caught exception when creating KinesisPartitionGroupOffset from offsetStr: " + offsetStr); - } - } - - @Override - public StreamPartitionMsgOffset create(StreamPartitionMsgOffset other) { - return new KinesisPartitionGroupOffset(((KinesisPartitionGroupOffset) other).getShardToStartSequenceMap()); + return new KinesisPartitionGroupOffset(offsetStr); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java index 12af4765fd03..e68c84e82e1d 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java @@ -18,20 +18,17 @@ */ package org.apache.pinot.plugin.stream.kinesis; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectReader; -import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Preconditions; -import java.io.IOException; import java.util.Map; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.apache.pinot.spi.utils.JsonUtils; + /** * A {@link StreamPartitionMsgOffset} implementation for the Kinesis partition group consumption - * A partition group consists of 1 or more shards. The KinesisCheckpoint maintains a Map of shards to the - * sequenceNumber. + * A partition group consists of 1 shard. The KinesisCheckpoint maintains the shardId and sequenceNumber. * The sequenceNumber is the id (equivalent to offset in kafka) for the messages in the shard. * From the Kinesis documentation: * Each data record has a sequence number that is unique per partition-key within its shard. @@ -41,69 +38,41 @@ * The longer the time period between write requests, the larger the sequence numbers become. */ public class KinesisPartitionGroupOffset implements StreamPartitionMsgOffset { - private static final ObjectMapper DEFAULT_MAPPER = new ObjectMapper(); - public static final ObjectReader DEFAULT_READER = DEFAULT_MAPPER.reader(); - public static final ObjectWriter DEFAULT_WRITER = DEFAULT_MAPPER.writer(); - - private final Map _shardToStartSequenceMap; - - public KinesisPartitionGroupOffset(Map shardToStartSequenceMap) { - _shardToStartSequenceMap = shardToStartSequenceMap; - } + private final String _shardId; + private final String _sequenceNumber; - public KinesisPartitionGroupOffset(String offsetStr) - throws IOException { - _shardToStartSequenceMap = stringToObject(offsetStr, new TypeReference>() { - }); + public KinesisPartitionGroupOffset(String shardId, String sequenceNumber) { + _shardId = shardId; + _sequenceNumber = sequenceNumber; } - public Map getShardToStartSequenceMap() { - return _shardToStartSequenceMap; - } - - @Override - public String toString() { + public KinesisPartitionGroupOffset(String offsetStr) { try { - return objectToString(_shardToStartSequenceMap); - } catch (JsonProcessingException e) { - throw new IllegalStateException( - "Caught exception when converting KinesisCheckpoint to string: " + _shardToStartSequenceMap); + ObjectNode objectNode = (ObjectNode) JsonUtils.stringToJsonNode(offsetStr); + Preconditions.checkArgument(objectNode.size() == 1); + Map.Entry entry = objectNode.fields().next(); + _shardId = entry.getKey(); + _sequenceNumber = entry.getValue().asText(); + } catch (Exception e) { + throw new IllegalArgumentException("Invalid Kinesis offset: " + offsetStr); } } - @Override - public KinesisPartitionGroupOffset fromString(String kinesisCheckpointStr) { - try { - return new KinesisPartitionGroupOffset(kinesisCheckpointStr); - } catch (IOException e) { - throw new IllegalStateException( - "Caught exception when converting string to KinesisCheckpoint: " + kinesisCheckpointStr); - } + public String getShardId() { + return _shardId; } - @Override - public int compareTo(StreamPartitionMsgOffset o) { - Preconditions.checkNotNull(o); - KinesisPartitionGroupOffset other = (KinesisPartitionGroupOffset) o; - Preconditions.checkNotNull(other._shardToStartSequenceMap); - Preconditions.checkNotNull(_shardToStartSequenceMap); - Preconditions - .checkState(other._shardToStartSequenceMap.size() == 1, "Only 1 shard per consumer supported. Found: %s", - other._shardToStartSequenceMap); - Preconditions - .checkState(_shardToStartSequenceMap.size() == 1, "Only 1 shard per consumer supported. Found: %s", - _shardToStartSequenceMap); - return _shardToStartSequenceMap.values().iterator().next() - .compareTo(other._shardToStartSequenceMap.values().iterator().next()); + public String getSequenceNumber() { + return _sequenceNumber; } - public static T stringToObject(String jsonString, TypeReference valueTypeRef) - throws IOException { - return DEFAULT_READER.forType(valueTypeRef).readValue(jsonString); + @Override + public String toString() { + return JsonUtils.newObjectNode().put(_shardId, _sequenceNumber).toString(); } - public static String objectToString(Object object) - throws JsonProcessingException { - return DEFAULT_WRITER.writeValueAsString(object); + @Override + public int compareTo(StreamPartitionMsgOffset other) { + return _sequenceNumber.compareTo(((KinesisPartitionGroupOffset) other)._sequenceNumber); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java deleted file mode 100644 index 3b6acb5542c6..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * 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.pinot.plugin.stream.kinesis; - -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; - - -/** - * A {@link MessageBatch} for collecting records from the Kinesis stream - */ -public class KinesisRecordsBatch implements MessageBatch { - private final List _recordList; - private final String _shardId; - private final boolean _endOfShard; - - public KinesisRecordsBatch(List recordList, String shardId, boolean endOfShard) { - _recordList = recordList; - _shardId = shardId; - _endOfShard = endOfShard; - } - - @Override - public int getMessageCount() { - return _recordList.size(); - } - - @Override - public KinesisStreamMessage getMessageAtIndex(int index) { - return _recordList.get(index); - } - - @Override - public byte[] getMessageBytesAtIndex(int index) { - return _recordList.get(index).getValue(); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_recordList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _recordList.get(index).getValue().length; - } - - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put(_shardId, _recordList.get(index).sequenceNumber()); - return new KinesisPartitionGroupOffset(shardToSequenceMap); - } - - @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isEndOfPartitionGroup() { - return _endOfShard; - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return _recordList.get(index).getMetadata(); - } - - @Override - public StreamMessage getStreamMessage(int index) { - return _recordList.get(index); - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java index 61c55efe6252..cbabaf608a09 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java @@ -24,15 +24,17 @@ import org.apache.pinot.spi.stream.StreamMessageMetadata; +// TODO: Make it a util class public class KinesisStreamMessageMetadata extends StreamMessageMetadata { public static final String APPRX_ARRIVAL_TIMESTAMP_KEY = "apprxArrivalTimestamp"; public static final String SEQUENCE_NUMBER_KEY = "sequenceNumber"; - public KinesisStreamMessageMetadata(long recordIngestionTimeMs, - @Nullable GenericRow headers) { + @Deprecated + public KinesisStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers) { super(recordIngestionTimeMs, headers); } + @Deprecated public KinesisStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { super(recordIngestionTimeMs, headers, metadata); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java index 96bc0502c6bd..4a6617ad0923 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.stream.ConsumerPartitionState; -import org.apache.pinot.spi.stream.MessageBatch; import org.apache.pinot.spi.stream.OffsetCriteria; import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; @@ -108,12 +107,12 @@ public List computePartitionGroupMetadata(String clientI for (PartitionGroupConsumptionStatus currentPartitionGroupConsumptionStatus : partitionGroupConsumptionStatuses) { KinesisPartitionGroupOffset kinesisStartCheckpoint = (KinesisPartitionGroupOffset) currentPartitionGroupConsumptionStatus.getStartOffset(); - String shardId = kinesisStartCheckpoint.getShardToStartSequenceMap().keySet().iterator().next(); + String shardId = kinesisStartCheckpoint.getShardId(); shardsInCurrent.add(shardId); Shard shard = shardIdToShardMap.get(shardId); if (shard == null) { // Shard has expired shardsEnded.add(shardId); - String lastConsumedSequenceID = kinesisStartCheckpoint.getShardToStartSequenceMap().get(shardId); + String lastConsumedSequenceID = kinesisStartCheckpoint.getSequenceNumber(); LOGGER.warn("Kinesis shard with id: " + shardId + " has expired. Data has been consumed from the shard till sequence number: " + lastConsumedSequenceID + ". There can be potential data loss."); @@ -145,7 +144,6 @@ public List computePartitionGroupMetadata(String clientI if (shardsInCurrent.contains(newShardId)) { continue; } - StreamPartitionMsgOffset newStartOffset; Shard newShard = entry.getValue(); String parentShardId = newShard.parentShardId(); @@ -155,9 +153,10 @@ public List computePartitionGroupMetadata(String clientI // 3. Parent reached EOL and completely consumed. if (parentShardId == null || !shardIdToShardMap.containsKey(parentShardId) || shardsEnded.contains( parentShardId)) { - Map shardToSequenceNumberMap = new HashMap<>(); - shardToSequenceNumberMap.put(newShardId, newShard.sequenceNumberRange().startingSequenceNumber()); - newStartOffset = new KinesisPartitionGroupOffset(shardToSequenceNumberMap); + // TODO: Revisit this. Kinesis starts consuming after the start sequence number, and we might miss the first + // message. + StreamPartitionMsgOffset newStartOffset = + new KinesisPartitionGroupOffset(newShardId, newShard.sequenceNumberRange().startingSequenceNumber()); int partitionGroupId = getPartitionGroupIdFromShardId(newShardId); newPartitionGroupMetadataList.add(new PartitionGroupMetadata(partitionGroupId, newStartOffset)); } @@ -179,30 +178,24 @@ private int getPartitionGroupIdFromShardId(String shardId) { private boolean consumedEndOfShard(StreamPartitionMsgOffset startCheckpoint, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) throws IOException, TimeoutException { - PartitionGroupConsumer partitionGroupConsumer = - _kinesisStreamConsumerFactory.createPartitionGroupConsumer(_clientId, partitionGroupConsumptionStatus); - - MessageBatch messageBatch; - try { - messageBatch = partitionGroupConsumer.fetchMessages(startCheckpoint, null, _fetchTimeoutMs); - } finally { - partitionGroupConsumer.close(); + try (PartitionGroupConsumer partitionGroupConsumer = _kinesisStreamConsumerFactory.createPartitionGroupConsumer( + _clientId, partitionGroupConsumptionStatus)) { + return partitionGroupConsumer.fetchMessages(startCheckpoint, _fetchTimeoutMs).isEndOfPartitionGroup(); } - return messageBatch.isEndOfPartitionGroup(); } @Override public Map getCurrentPartitionLagState( Map currentPartitionStateMap) { Map perPartitionLag = new HashMap<>(); - for (Map.Entry entry: currentPartitionStateMap.entrySet()) { + for (Map.Entry entry : currentPartitionStateMap.entrySet()) { ConsumerPartitionState partitionState = entry.getValue(); // Compute record-availability String recordAvailabilityLag = "UNKNOWN"; RowMetadata lastProcessedMessageMetadata = partitionState.getLastProcessedRowMetadata(); if (lastProcessedMessageMetadata != null && partitionState.getLastProcessedTimeMs() > 0) { - long availabilityLag = partitionState.getLastProcessedTimeMs() - - lastProcessedMessageMetadata.getRecordIngestionTimeMs(); + long availabilityLag = + partitionState.getLastProcessedTimeMs() - lastProcessedMessageMetadata.getRecordIngestionTimeMs(); recordAvailabilityLag = String.valueOf(availabilityLag); } perPartitionLag.put(entry.getKey(), new KinesisConsumerPartitionLag(recordAvailabilityLag)); @@ -211,5 +204,6 @@ public Map getCurrentPartitionLagState( } @Override - public void close() { } + public void close() { + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java index 5d620de604e1..04a8f114cc92 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java @@ -25,9 +25,7 @@ import java.util.Map; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.easymock.Capture; -import org.testng.Assert; +import org.mockito.ArgumentCaptor; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import software.amazon.awssdk.core.SdkBytes; @@ -40,10 +38,11 @@ import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class KinesisConsumerTest { @@ -58,19 +57,17 @@ public class KinesisConsumerTest { private static final String PLACEHOLDER = "DUMMY"; private static final int MAX_RECORDS_TO_FETCH = 20; - private KinesisConnectionHandler _kinesisConnectionHandler; - private StreamConsumerFactory _streamConsumerFactory; private KinesisClient _kinesisClient; - private List _recordList; + private KinesisConfig _kinesisConfig; + private List _records; private KinesisConfig getKinesisConfig() { Map props = new HashMap<>(); props.put(StreamConfigProperties.STREAM_TYPE, STREAM_TYPE); props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_TOPIC_NAME), STREAM_NAME); - props.put(StreamConfigProperties - .constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_CONSUMER_FACTORY_CLASS), - KinesisConsumerFactory.class.getName()); + props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, + StreamConfigProperties.STREAM_CONSUMER_FACTORY_CLASS), KinesisConsumerFactory.class.getName()); props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_DECODER_CLASS), "org.apache.pinot.plugin.inputformat.json.JSONMessageDecoder"); props.put(KinesisConfig.REGION, AWS_REGION); @@ -81,118 +78,78 @@ private KinesisConfig getKinesisConfig() { @BeforeMethod public void setupTest() { - _kinesisConnectionHandler = createMock(KinesisConnectionHandler.class); - _kinesisClient = createMock(KinesisClient.class); - _streamConsumerFactory = createMock(StreamConsumerFactory.class); - - _recordList = new ArrayList<>(); - + _kinesisClient = mock(KinesisClient.class); + _kinesisConfig = getKinesisConfig(); + _records = new ArrayList<>(NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { Record record = Record.builder().data(SdkBytes.fromUtf8String(DUMMY_RECORD_PREFIX + i)).partitionKey(PARTITION_KEY_PREFIX + i) - .approximateArrivalTimestamp(Instant.now()) - .sequenceNumber(String.valueOf(i + 1)).build(); - _recordList.add(record); + .approximateArrivalTimestamp(Instant.now()).sequenceNumber(String.valueOf(i + 1)).build(); + _records.add(record); } } @Test public void testBasicConsumer() { - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); + ArgumentCaptor getRecordsRequestCapture = ArgumentCaptor.forClass(GetRecordsRequest.class); + ArgumentCaptor getShardIteratorRequestCapture = + ArgumentCaptor.forClass(GetShardIteratorRequest.class); GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(null).records(_recordList).build(); + GetRecordsResponse.builder().nextShardIterator(null).records(_records).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); + when(_kinesisClient.getRecords(getRecordsRequestCapture.capture())).thenReturn(getRecordsResponse); + when(_kinesisClient.getShardIterator(getShardIteratorRequestCapture.capture())).thenReturn( + getShardIteratorResponse); - KinesisConsumer kinesisConsumer = new KinesisConsumer(getKinesisConfig(), _kinesisClient); + KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); + KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); + KinesisMessageBatch kinesisMessageBatch = kinesisConsumer.fetchMessages(startOffset, TIMEOUT); - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); - - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), NUM_RECORDS); + assertEquals(kinesisMessageBatch.getMessageCount(), NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); + assertEquals(baToString(kinesisMessageBatch.getStreamMessage(i).getValue()), DUMMY_RECORD_PREFIX + i); } - Assert.assertFalse(kinesisRecordsBatch.isEndOfPartitionGroup()); - } - - @Test - public void testBasicConsumerWithMaxRecordsLimit() { - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); - - GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(PLACEHOLDER).records(_recordList).build(); - GetShardIteratorResponse getShardIteratorResponse = - GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); - - KinesisConfig kinesisConfig = getKinesisConfig(); - KinesisConsumer kinesisConsumer = new KinesisConsumer(kinesisConfig, _kinesisClient); - - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); - - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), MAX_RECORDS_TO_FETCH); - - for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); - } + assertFalse(kinesisMessageBatch.isEndOfPartitionGroup()); + assertEquals(getRecordsRequestCapture.getValue().shardIterator(), "DUMMY"); + assertEquals(getShardIteratorRequestCapture.getValue().shardId(), "0"); } @Test public void testBasicConsumerWithChildShard() { - List shardList = new ArrayList<>(); shardList.add(ChildShard.builder().shardId(PLACEHOLDER).parentShards("0").build()); - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); + ArgumentCaptor getRecordsRequestCapture = ArgumentCaptor.forClass(GetRecordsRequest.class); + ArgumentCaptor getShardIteratorRequestCapture = + ArgumentCaptor.forClass(GetShardIteratorRequest.class); GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(null).records(_recordList).childShards(shardList).build(); + GetRecordsResponse.builder().nextShardIterator(null).records(_records).childShards(shardList).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); + when(_kinesisClient.getRecords(getRecordsRequestCapture.capture())).thenReturn(getRecordsResponse); + when(_kinesisClient.getShardIterator(getShardIteratorRequestCapture.capture())).thenReturn( + getShardIteratorResponse); - KinesisConfig kinesisConfig = getKinesisConfig(); - KinesisConsumer kinesisConsumer = new KinesisConsumer(kinesisConfig, _kinesisClient); + KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); + KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); + KinesisMessageBatch kinesisMessageBatch = kinesisConsumer.fetchMessages(startOffset, TIMEOUT); - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); - - Assert.assertTrue(kinesisRecordsBatch.isEndOfPartitionGroup()); - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), NUM_RECORDS); + assertTrue(kinesisMessageBatch.isEndOfPartitionGroup()); + assertEquals(kinesisMessageBatch.getMessageCount(), NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); + assertEquals(baToString(kinesisMessageBatch.getStreamMessage(i).getValue()), DUMMY_RECORD_PREFIX + i); } + + assertEquals(getRecordsRequestCapture.getValue().shardIterator(), "DUMMY"); + assertEquals(getShardIteratorRequestCapture.getValue().shardId(), "0"); } public String baToString(byte[] bytes) { diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java new file mode 100644 index 000000000000..4e64dae54961 --- /dev/null +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java @@ -0,0 +1,84 @@ +/** + * 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.pinot.plugin.stream.kinesis; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.StreamMessageMetadata; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + + +public class KinesisMessageBatchTest { + private static final String SHARD_ID = "shard-0000000000"; + + @Test + public void testMessageBatch() { + int numMessages = 5; + long baseTimeMs = System.currentTimeMillis(); + List messages = new ArrayList<>(numMessages); + for (int i = 0; i < numMessages; i++) { + messages.add(createStreamMessage(i, "key-" + i, "value-" + i, baseTimeMs + i)); + } + KinesisMessageBatch batch = + new KinesisMessageBatch(messages, new KinesisPartitionGroupOffset(SHARD_ID, Integer.toString(numMessages - 1)), + false); + + for (int i = 0; i < numMessages; i++) { + BytesStreamMessage streamMessage = batch.getStreamMessage(i); + byte[] key = streamMessage.getKey(); + assertNotNull(key); + assertEquals(new String(key, StandardCharsets.UTF_8), "key-" + i); + byte[] value = streamMessage.getValue(); + assertEquals(new String(value, StandardCharsets.UTF_8), "value-" + i); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), baseTimeMs + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof KinesisPartitionGroupOffset); + assertEquals(((KinesisPartitionGroupOffset) offset).getShardId(), SHARD_ID); + assertEquals(((KinesisPartitionGroupOffset) offset).getSequenceNumber(), Integer.toString(i)); + assertSame(metadata.getNextOffset(), offset); + } + + // Batch level operations + assertEquals(batch.getMessageCount(), numMessages); + assertEquals(batch.getUnfilteredMessageCount(), numMessages); + assertFalse(batch.isEndOfPartitionGroup()); + StreamPartitionMsgOffset nextBatchOffset = batch.getOffsetOfNextBatch(); + assertTrue(nextBatchOffset instanceof KinesisPartitionGroupOffset); + assertEquals(((KinesisPartitionGroupOffset) nextBatchOffset).getShardId(), SHARD_ID); + assertEquals(((KinesisPartitionGroupOffset) nextBatchOffset).getSequenceNumber(), + Integer.toString(numMessages - 1)); + } + + private static BytesStreamMessage createStreamMessage(int sequenceNumber, String key, String value, + long recordIngestionTimeMs) { + KinesisPartitionGroupOffset offset = new KinesisPartitionGroupOffset(SHARD_ID, Integer.toString(sequenceNumber)); + StreamMessageMetadata metadata = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(recordIngestionTimeMs).setOffset(offset, offset) + .build(); + return new BytesStreamMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8), + metadata); + } +} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java deleted file mode 100644 index 83d7ded36e14..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * 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.pinot.plugin.stream.kinesis; - -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class KinesisRecordsBatchTest { - private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; - - @Test - public void testMessageBatchAPIs() { - int msgCount = 5; - long baseTimeMs = System.currentTimeMillis(); - List msgList = new ArrayList<>(msgCount); - - - for (int i = 0; i < msgCount; i++) { - msgList.add(createStreamMessage(i, "key-" + i, "value-" + i, baseTimeMs + i)); - } - KinesisRecordsBatch batch = new KinesisRecordsBatch(msgList, "shard-0000000000", false); - - for (int i = 0; i < msgCount; i++) { - Assert.assertEquals(batch.getMessageLengthAtIndex(i), 7); // length of characters in "value-$i" - Assert.assertEquals(batch.getMessageOffsetAtIndex(i), 0); - - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = - (KinesisPartitionGroupOffset) batch.getNextStreamPartitionMsgOffsetAtIndex(i); - Assert.assertNotNull(kinesisPartitionGroupOffset); - Assert.assertNotNull(kinesisPartitionGroupOffset.getShardToStartSequenceMap().get("shard-0000000000"), - String.valueOf(i)); // why is "next" stream partition msg offset return the exact offset at index i ? - - KinesisStreamMessageMetadata metadata = (KinesisStreamMessageMetadata) batch.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), baseTimeMs + i); - Assert.assertEquals(batch.getMessageBytesAtIndex(i), ("value-" + i).getBytes(StandardCharsets.UTF_8)); - } - - // Batch level operations - Assert.assertEquals(batch.getMessageCount(), msgCount); - Assert.assertFalse(batch.isEndOfPartitionGroup()); - - KinesisPartitionGroupOffset nextBatchOffset = (KinesisPartitionGroupOffset) batch.getOffsetOfNextBatch(); - Assert.assertNotNull(nextBatchOffset); - Assert.assertNotNull(nextBatchOffset.getShardToStartSequenceMap().get("shard-0000000000"), - String.valueOf(msgCount - 1)); // default implementation doesn't make sense ?? - Assert.assertEquals(batch.getUnfilteredMessageCount(), msgCount); // always size of the batch for Kinesis - - // unsupported operations for kinesis batch - Assert.assertThrows(UnsupportedOperationException.class, () -> { - batch.getNextStreamMessageOffsetAtIndex(0); - }); - - // batch.getMessageAtIndex(i); // deprecated - } - - private static KinesisStreamMessage createStreamMessage(int sequenceNumber, String key, String value, - long recordIngestionTimeMs) { - KinesisStreamMessageMetadata metadata = new KinesisStreamMessageMetadata(recordIngestionTimeMs, null); - byte[] valueArray = value != null ? value.getBytes(StandardCharsets.UTF_8) : EMPTY_BYTE_ARRAY; - return new KinesisStreamMessage( - key != null ? key.getBytes(StandardCharsets.UTF_8) : EMPTY_BYTE_ARRAY, valueArray, - String.valueOf(sequenceNumber), metadata, valueArray.length); - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java index 8d9288e2cf92..f536fe78d801 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java @@ -30,8 +30,7 @@ import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.easymock.Capture; -import org.easymock.CaptureType; +import org.mockito.ArgumentCaptor; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -39,7 +38,8 @@ import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; -import static org.easymock.EasyMock.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class KinesisStreamMetadataProviderTest { @@ -71,9 +71,9 @@ private StreamConfig getStreamConfig() { @BeforeMethod public void setupTest() { - _kinesisConnectionHandler = createMock(KinesisConnectionHandler.class); - _streamConsumerFactory = createMock(StreamConsumerFactory.class); - _partitionGroupConsumer = createNiceMock(PartitionGroupConsumer.class); + _kinesisConnectionHandler = mock(KinesisConnectionHandler.class); + _streamConsumerFactory = mock(StreamConsumerFactory.class); + _partitionGroupConsumer = mock(PartitionGroupConsumer.class); _kinesisStreamMetadataProvider = new KinesisStreamMetadataProvider(CLIENT_ID, getStreamConfig(), _kinesisConnectionHandler, _streamConsumerFactory); @@ -87,11 +87,11 @@ public void getPartitionsGroupInfoListTest() Shard shard1 = Shard.builder().shardId(SHARD_ID_1) .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - replay(_kinesisConnectionHandler); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), new ArrayList<>(), TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), new ArrayList<>(), + TIMEOUT); Assert.assertEquals(result.size(), 2); Assert.assertEquals(result.get(0).getPartitionGroupId(), 0); @@ -103,37 +103,35 @@ public void getPartitionsGroupInfoEndOfShardTest() throws Exception { List currentPartitionGroupMeta = new ArrayList<>(); - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); + KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("0", "1"); - currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, - kinesisPartitionGroupOffset, "CONSUMING")); + currentPartitionGroupMeta.add( + new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, + "CONSUMING")); - Capture checkpointArgs = newCapture(CaptureType.ALL); - Capture partitionGroupMetadataCapture = newCapture(CaptureType.ALL); - Capture intArguments = newCapture(CaptureType.ALL); - Capture stringCapture = newCapture(CaptureType.ALL); + ArgumentCaptor checkpointArgs = ArgumentCaptor.forClass(StreamPartitionMsgOffset.class); + ArgumentCaptor partitionGroupMetadataCapture = + ArgumentCaptor.forClass(PartitionGroupConsumptionStatus.class); + ArgumentCaptor intArguments = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor stringCapture = ArgumentCaptor.forClass(String.class); Shard shard0 = Shard.builder().shardId(SHARD_ID_0).sequenceNumberRange( SequenceNumberRange.builder().startingSequenceNumber("1").endingSequenceNumber("1").build()).build(); - Shard shard1 = Shard.builder().shardId(SHARD_ID_1).sequenceNumberRange( - SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - expect(_streamConsumerFactory - .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) - .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer - .fetchMessages(capture(checkpointArgs), capture(checkpointArgs), captureInt(intArguments))) - .andReturn(new KinesisRecordsBatch(new ArrayList<>(), "0", true)).anyTimes(); - - replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); + Shard shard1 = Shard.builder().shardId(SHARD_ID_1) + .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); + when(_streamConsumerFactory.createPartitionGroupConsumer(stringCapture.capture(), + partitionGroupMetadataCapture.capture())).thenReturn(_partitionGroupConsumer); + when(_partitionGroupConsumer.fetchMessages(checkpointArgs.capture(), intArguments.capture())).thenReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), currentPartitionGroupMeta, TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), + currentPartitionGroupMeta, TIMEOUT); Assert.assertEquals(result.size(), 1); Assert.assertEquals(result.get(0).getPartitionGroupId(), 1); + Assert.assertEquals(partitionGroupMetadataCapture.getValue().getSequenceNumber(), 1); } @Test @@ -143,35 +141,35 @@ public void getPartitionsGroupInfoChildShardsest() Map shardToSequenceMap = new HashMap<>(); shardToSequenceMap.put("1", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); + KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("1", "1"); - currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, - kinesisPartitionGroupOffset, "CONSUMING")); + currentPartitionGroupMeta.add( + new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, + "CONSUMING")); - Capture checkpointArgs = newCapture(CaptureType.ALL); - Capture partitionGroupMetadataCapture = newCapture(CaptureType.ALL); - Capture intArguments = newCapture(CaptureType.ALL); - Capture stringCapture = newCapture(CaptureType.ALL); + ArgumentCaptor checkpointArgs = ArgumentCaptor.forClass(StreamPartitionMsgOffset.class); + ArgumentCaptor partitionGroupMetadataCapture = + ArgumentCaptor.forClass(PartitionGroupConsumptionStatus.class); + ArgumentCaptor intArguments = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor stringCapture = ArgumentCaptor.forClass(String.class); - Shard shard0 = Shard.builder().shardId(SHARD_ID_0).parentShardId(SHARD_ID_1).sequenceNumberRange( - SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); + Shard shard0 = Shard.builder().shardId(SHARD_ID_0).parentShardId(SHARD_ID_1) + .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); Shard shard1 = Shard.builder().shardId(SHARD_ID_1).sequenceNumberRange( SequenceNumberRange.builder().startingSequenceNumber("1").endingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - expect(_streamConsumerFactory - .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) - .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer - .fetchMessages(capture(checkpointArgs), capture(checkpointArgs), captureInt(intArguments))) - .andReturn(new KinesisRecordsBatch(new ArrayList<>(), "0", true)).anyTimes(); - - replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); + when(_streamConsumerFactory.createPartitionGroupConsumer(stringCapture.capture(), + partitionGroupMetadataCapture.capture())).thenReturn(_partitionGroupConsumer); + when(_partitionGroupConsumer.fetchMessages(checkpointArgs.capture(), intArguments.capture())).thenReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), currentPartitionGroupMeta, TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), + currentPartitionGroupMeta, TIMEOUT); Assert.assertEquals(result.size(), 1); Assert.assertEquals(result.get(0).getPartitionGroupId(), 0); + Assert.assertEquals(partitionGroupMetadataCapture.getValue().getSequenceNumber(), 1); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml index c099839ec5e3..2037493483d2 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml @@ -38,8 +38,8 @@ package ${basedir}/../../.. 0.16.0 - 1.60.1 - 1.62.2 + 1.63.0 + 1.63.0 2.6.2 1.17 @@ -51,6 +51,11 @@ 1.19.7 test + + org.mockito + mockito-core + test + org.apache.pulsar pulsar-client-original diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java index d8e07658e1fa..b322d5c084a1 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java @@ -18,35 +18,26 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import java.io.IOException; import org.apache.commons.codec.binary.Hex; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pulsar.client.api.MessageId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * {@link StreamPartitionMsgOffset} implementation for Pulsar {@link MessageId} */ public class MessageIdStreamOffset implements StreamPartitionMsgOffset { - private static final Logger LOGGER = LoggerFactory.getLogger(MessageIdStreamOffset.class); - private MessageId _messageId; + private final MessageId _messageId; public MessageIdStreamOffset(MessageId messageId) { _messageId = messageId; } - /** - * returns the class object from string message id in the format ledgerId:entryId:partitionId - * throws {@link IOException} if message if format is invalid. - * @param messageId - */ public MessageIdStreamOffset(String messageId) { try { _messageId = MessageId.fromByteArray(Hex.decodeHex(messageId)); } catch (Exception e) { - LOGGER.warn("Cannot parse message id " + messageId, e); + throw new IllegalArgumentException("Invalid Pulsar message id: " + messageId); } } @@ -54,15 +45,9 @@ public MessageId getMessageId() { return _messageId; } - @Override - public StreamPartitionMsgOffset fromString(String streamPartitionMsgOffsetStr) { - return new MessageIdStreamOffset(streamPartitionMsgOffsetStr); - } - @Override public int compareTo(StreamPartitionMsgOffset other) { - MessageIdStreamOffset messageIdStreamOffset = (MessageIdStreamOffset) other; - return _messageId.compareTo(messageIdStreamOffset.getMessageId()); + return _messageId.compareTo(((MessageIdStreamOffset) other).getMessageId()); } @Override diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java index 69c1c5323858..74009048d596 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java @@ -27,21 +27,13 @@ * {@link StreamPartitionMsgOffsetFactory} implementation for Pulsar streams. */ public class MessageIdStreamOffsetFactory implements StreamPartitionMsgOffsetFactory { - private StreamConfig _streamConfig; @Override public void init(StreamConfig streamConfig) { - _streamConfig = streamConfig; } @Override public StreamPartitionMsgOffset create(String offsetStr) { return new MessageIdStreamOffset(offsetStr); } - - @Override - public StreamPartitionMsgOffset create(StreamPartitionMsgOffset other) { - MessageIdStreamOffset messageIdStreamOffset = (MessageIdStreamOffset) other; - return new MessageIdStreamOffset(messageIdStreamOffset.getMessageId()); - } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java index 637899d649a6..8011e2c14843 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java @@ -30,10 +30,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.spi.stream.OffsetCriteria; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -46,17 +44,15 @@ public class PulsarConfig { public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; public static final String AUTHENTICATION_TOKEN = "authenticationToken"; public static final String TLS_TRUST_CERTS_FILE_PATH = "tlsTrustCertsFilePath"; - public static final String OAUTH_ISSUER_URL = "issuerUrl"; public static final String OAUTH_CREDS_FILE_PATH = "credsFilePath"; public static final String OAUTH_AUDIENCE = "audience"; public static final String ENABLE_KEY_VALUE_STITCH = "enableKeyValueStitch"; public static final String METADATA_FIELDS = "metadata.fields"; //list of the metadata fields comma separated - private final String _pulsarTopicName; private final String _subscriberId; + private final String _pulsarTopicName; private final String _bootstrapServers; - private final MessageId _initialMessageId; private final SubscriptionInitialPosition _subscriptionInitialPosition; private final String _authenticationToken; private final String _tlsTrustCertsFilePath; @@ -74,6 +70,7 @@ public class PulsarConfig { private final boolean _enableKeyValueStitch; private final boolean _populateMetadata; private final Set _metadataFields; + public PulsarConfig(StreamConfig streamConfig, String subscriberId) { Map streamConfigMap = streamConfig.getStreamConfigsMap(); _subscriberId = subscriberId; @@ -82,28 +79,24 @@ public PulsarConfig(StreamConfig streamConfig, String subscriberId) { _bootstrapServers = getConfigValue(streamConfigMap, BOOTSTRAP_SERVERS); Preconditions.checkNotNull(_bootstrapServers, "No brokers provided in the config"); + _subscriptionInitialPosition = PulsarUtils.offsetCriteriaToSubscription(streamConfig.getOffsetCriteria()); _authenticationToken = getConfigValue(streamConfigMap, AUTHENTICATION_TOKEN); _tlsTrustCertsFilePath = getConfigValue(streamConfigMap, TLS_TRUST_CERTS_FILE_PATH); - _enableKeyValueStitch = Boolean.parseBoolean(getConfigValue(streamConfigMap, ENABLE_KEY_VALUE_STITCH)); - - OffsetCriteria offsetCriteria = streamConfig.getOffsetCriteria(); + _issuerUrl = getConfigValue(streamConfigMap, OAUTH_ISSUER_URL); + _credentialsFilePath = getConfigValue(streamConfigMap, OAUTH_CREDS_FILE_PATH); + if (StringUtils.isNotBlank(_credentialsFilePath)) { + validateOAuthCredFile(); + } + _audience = getConfigValue(streamConfigMap, OAUTH_AUDIENCE); - _subscriptionInitialPosition = PulsarUtils.offsetCriteriaToSubscription(offsetCriteria); - _initialMessageId = PulsarUtils.offsetCriteriaToMessageId(offsetCriteria); - _populateMetadata = Boolean.parseBoolean(getConfigValueOrDefault(streamConfigMap, - StreamConfigProperties.METADATA_POPULATE, "false")); + _enableKeyValueStitch = Boolean.parseBoolean(getConfigValue(streamConfigMap, ENABLE_KEY_VALUE_STITCH)); + _populateMetadata = Boolean.parseBoolean(getConfigValue(streamConfigMap, StreamConfigProperties.METADATA_POPULATE)); String metadataFieldsToExtractCSV = getConfigValueOrDefault(streamConfigMap, METADATA_FIELDS, ""); if (StringUtils.isBlank(metadataFieldsToExtractCSV) || !_populateMetadata) { _metadataFields = Collections.emptySet(); } else { _metadataFields = parseConfigStringToEnumSet(metadataFieldsToExtractCSV); } - _issuerUrl = getConfigValue(streamConfigMap, OAUTH_ISSUER_URL); - _credentialsFilePath = getConfigValue(streamConfigMap, OAUTH_CREDS_FILE_PATH); - if (StringUtils.isNotBlank(_credentialsFilePath)) { - validateOAuthCredFile(); - } - _audience = getConfigValue(streamConfigMap, OAUTH_AUDIENCE); } @VisibleForTesting @@ -111,13 +104,13 @@ private void validateOAuthCredFile() { try { URL credFilePathUrl = new URL(_credentialsFilePath); if (!"file".equals(credFilePathUrl.getProtocol())) { - throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath - + ". URL protocol must be file://"); + throw new IllegalArgumentException( + "Invalid credentials file path: " + _credentialsFilePath + ". URL protocol must be file://"); } File credFile = new File(credFilePathUrl.getPath()); if (!credFile.exists()) { - throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath - + ". File does not exist."); + throw new IllegalArgumentException( + "Invalid credentials file path: " + _credentialsFilePath + ". File does not exist."); } } catch (MalformedURLException mue) { throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath, mue); @@ -136,16 +129,14 @@ private Set parseConfigS String listOfMetadataFields) { try { String[] metadataFieldsArr = listOfMetadataFields.split(","); - return Stream.of(metadataFieldsArr) - .map(String::trim) - .filter(StringUtils::isNotBlank) - .map(PulsarStreamMessageMetadata.PulsarMessageMetadataValue::findByKey) - .filter(Objects::nonNull) + return Stream.of(metadataFieldsArr).map(String::trim).filter(StringUtils::isNotBlank) + .map(PulsarStreamMessageMetadata.PulsarMessageMetadataValue::findByKey).filter(Objects::nonNull) .collect(Collectors.toSet()); } catch (Exception e) { throw new IllegalArgumentException("Invalid metadata fields list: " + listOfMetadataFields, e); } } + public String getPulsarTopicName() { return _pulsarTopicName; } @@ -158,10 +149,6 @@ public String getBootstrapServers() { return _bootstrapServers; } - public MessageId getInitialMessageId() { - return _initialMessageId; - } - public SubscriptionInitialPosition getInitialSubscriberPosition() { return _subscriptionInitialPosition; } @@ -173,16 +160,6 @@ public String getAuthenticationToken() { public String getTlsTrustCertsFilePath() { return _tlsTrustCertsFilePath; } - public boolean getEnableKeyValueStitch() { - return _enableKeyValueStitch; - } - public boolean isPopulateMetadata() { - return _populateMetadata; - } - - public Set getMetadataFields() { - return _metadataFields; - } public String getIssuerUrl() { return _issuerUrl; @@ -195,4 +172,16 @@ public String getCredentialsFilePath() { public String getAudience() { return _audience; } + + public boolean getEnableKeyValueStitch() { + return _enableKeyValueStitch; + } + + public boolean isPopulateMetadata() { + return _populateMetadata; + } + + public Set getMetadataFields() { + return _metadataFields; + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java index 6614bd321d10..65eb583df86e 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java @@ -20,7 +20,6 @@ import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -30,10 +29,6 @@ * A {@link StreamConsumerFactory} implementation for the Pulsar stream */ public class PulsarConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - throw new UnsupportedOperationException("Partition Level consumer is deprecated!"); - } @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { @@ -53,6 +48,7 @@ public StreamPartitionMsgOffsetFactory createStreamMsgOffsetFactory() { @Override public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { - return new PulsarPartitionLevelConsumer(clientId, _streamConfig, partitionGroupConsumptionStatus); + return new PulsarPartitionLevelConsumer(clientId, _streamConfig, + partitionGroupConsumptionStatus.getPartitionGroupId()); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java index 15e7a4c1e2a9..10ea7ca9d6d1 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java @@ -18,117 +18,43 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.internal.DefaultImplementation; /** * A {@link MessageBatch} for collecting messages from pulsar topic - * When 'enableKeyValueStitch' flag is enabled, existing {@link org.apache.pinot.spi.stream.StreamMessageDecoder} - * plugins will not work. A custom decoder will be needed to unpack key and value byte arrays and decode - * them independently. */ -public class PulsarMessageBatch implements MessageBatch { - private final List _messageList = new ArrayList<>(); - private final boolean _enableKeyValueStitch; +public class PulsarMessageBatch implements MessageBatch { + private final List _messages; + private final MessageIdStreamOffset _offsetOfNextBatch; + private final boolean _endOfTopic; - public PulsarMessageBatch(Iterable iterable, boolean enableKeyValueStitch) { - iterable.forEach(_messageList::add); - _enableKeyValueStitch = enableKeyValueStitch; + public PulsarMessageBatch(List messages, MessageIdStreamOffset offsetOfNextBatch, + boolean endOfTopic) { + _messages = messages; + _offsetOfNextBatch = offsetOfNextBatch; + _endOfTopic = endOfTopic; } @Override public int getMessageCount() { - return _messageList.size(); + return _messages.size(); } - @Deprecated @Override - public PulsarStreamMessage getMessageAtIndex(int index) { - return getMessageList(index); + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); } @Override - public StreamMessage getStreamMessage(int index) { - return getMessageList(index); - } - - private PulsarStreamMessage getMessageList(int index) { - return _messageList.get(index); - } - @Override - public byte[] getMessageBytesAtIndex(int index) { - return getMessageList(index).getValue(); - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return getMessageList(index).getMetadata(); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_messageList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageList.get(index).getValue().length; //if _enableKeyValueStitch is true, - // then they are already stitched in the consumer. If false, then the value is the raw value - } - - /** - * Returns next message id supposed to be present in the pulsar topic partition. - * The message id is composed of 3 parts - ledgerId, entryId and partitionId. - * The ledger id are always increasing in number but may not be sequential. e.g. for first 10 records ledger id can - * be 12 but for next 10 it can be 18. - * each entry inside a ledger is always in a sequential and increases by 1 for next message. - * the partition id is fixed for a particular partition. - * We return entryId incremented by 1 while keeping ledgerId and partitionId as same. - * If ledgerId has incremented, the {@link org.apache.pulsar.client.api.Reader} takes care of that during seek - * operation - * and returns the first record in the new ledger. - */ - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - MessageIdImpl currentMessageId = MessageIdImpl.convertToMessageIdImpl(_messageList.get(index).getMessageId()); - MessageId nextMessageId; - - long currentLedgerId = currentMessageId.getLedgerId(); - long currentEntryId = currentMessageId.getEntryId(); - int currentPartitionIndex = currentMessageId.getPartitionIndex(); - - if (currentMessageId instanceof BatchMessageIdImpl) { - int currentBatchIndex = ((BatchMessageIdImpl) currentMessageId).getBatchIndex(); - int currentBatchSize = ((BatchMessageIdImpl) currentMessageId).getBatchSize(); - - if (currentBatchIndex < currentBatchSize - 1) { - nextMessageId = - new BatchMessageIdImpl(currentLedgerId, currentEntryId, currentPartitionIndex, currentBatchIndex + 1, - currentBatchSize, ((BatchMessageIdImpl) currentMessageId).getAcker()); - } else { - nextMessageId = - new BatchMessageIdImpl(currentLedgerId, currentEntryId + 1, currentPartitionIndex, 0, currentBatchSize, - ((BatchMessageIdImpl) currentMessageId).getAcker()); - } - } else { - nextMessageId = DefaultImplementation.getDefaultImplementation() - .newMessageId(currentLedgerId, currentEntryId + 1, currentPartitionIndex); - } - return new MessageIdStreamOffset(nextMessageId); + public MessageIdStreamOffset getOffsetOfNextBatch() { + return _offsetOfNextBatch; } @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("Pulsar does not support long stream offsets"); + public boolean isEndOfPartitionGroup() { + return _endOfTopic; } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java deleted file mode 100644 index 6b208af49504..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * 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.pinot.plugin.stream.pulsar; - -import java.util.Base64; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pulsar.client.api.Message; - -public interface PulsarMetadataExtractor { - static PulsarMetadataExtractor build(boolean populateMetadata, - Set metadataValuesToExtract) { - return message -> { - long publishTime = message.getPublishTime(); - long brokerPublishTime = message.getBrokerPublishTime().orElse(0L); - long recordTimestamp = brokerPublishTime != 0 ? brokerPublishTime : publishTime; - - Map metadataMap = populateMetadataMap(populateMetadata, message, metadataValuesToExtract); - - GenericRow headerGenericRow = populateMetadata ? buildGenericRow(message) : null; - return new PulsarStreamMessageMetadata(recordTimestamp, headerGenericRow, metadataMap); - }; - } - - RowMetadata extract(Message record); - - static GenericRow buildGenericRow(Message message) { - if (MapUtils.isEmpty(message.getProperties())) { - return null; - } - GenericRow genericRow = new GenericRow(); - for (Map.Entry entry : message.getProperties().entrySet()) { - genericRow.putValue(entry.getKey(), entry.getValue()); - } - return genericRow; - } - - static Map populateMetadataMap(boolean populateAllFields, Message message, - Set metadataValuesToExtract) { - - Map metadataMap = new HashMap<>(); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, message, metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, message, metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, message, - metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, message, metadataMap); - - // Populate some timestamps for lag calculation even if populateMetadata is false - - if (!populateAllFields) { - return metadataMap; - } - - for (PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue : metadataValuesToExtract) { - populateMetadataField(metadataValue, message, metadataMap); - } - - return metadataMap; - } - - static void populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue value, - Message message, Map metadataMap) { - switch (value) { - case PUBLISH_TIME: - long publishTime = message.getPublishTime(); - if (publishTime > 0) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, - publishTime); - } - break; - case EVENT_TIME: - long eventTime = message.getEventTime(); - if (eventTime > 0) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, - eventTime); - } - break; - case BROKER_PUBLISH_TIME: - message.getBrokerPublishTime() - .ifPresent(brokerPublishTime -> setMetadataMapField(metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, brokerPublishTime)); - break; - case MESSAGE_KEY: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, - message.getKey()); - break; - case MESSAGE_ID: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, - message.getMessageId().toString()); - break; - case MESSAGE_ID_BYTES_B64: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, - message.getMessageId().toByteArray()); - break; - case PRODUCER_NAME: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PRODUCER_NAME, - message.getProducerName()); - break; - case SCHEMA_VERSION: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SCHEMA_VERSION, - message.getSchemaVersion()); - break; - case SEQUENCE_ID: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SEQUENCE_ID, - message.getSequenceId()); - break; - case ORDERING_KEY: - if (message.hasOrderingKey()) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.ORDERING_KEY, - message.getOrderingKey()); - } - break; - case SIZE: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SIZE, - message.size()); - break; - case TOPIC_NAME: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME, - message.getTopicName()); - break; - case INDEX: - message.getIndex().ifPresent(index -> setMetadataMapField(metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.INDEX, index)); - break; - case REDELIVERY_COUNT: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.REDELIVERY_COUNT, - message.getRedeliveryCount()); - break; - default: - throw new IllegalArgumentException("Unsupported metadata value: " + value); - } - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - String value) { - if (StringUtils.isNotBlank(value)) { - metadataMap.put(metadataValue.getKey(), value); - } - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - int value) { - setMetadataMapField(metadataMap, metadataValue, String.valueOf(value)); - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - long value) { - setMetadataMapField(metadataMap, metadataValue, String.valueOf(value)); - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - byte[] value) { - if (value != null && value.length > 0) { - setMetadataMapField(metadataMap, metadataValue, Base64.getEncoder().encodeToString(value)); - } - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java index 53b27eb963f0..673ea0c0e82c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java @@ -18,109 +18,61 @@ */ package org.apache.pinot.plugin.stream.pulsar; +import java.io.Closeable; import java.io.IOException; -import java.net.MalformedURLException; import java.net.URL; -import java.util.List; -import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.impl.auth.oauth2.AuthenticationFactoryOAuth2; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Manages the Pulsar client connection, given the partition id and {@link PulsarConfig} */ -public class PulsarPartitionLevelConnectionHandler { - private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConnectionHandler.class); - +public class PulsarPartitionLevelConnectionHandler implements Closeable { protected final PulsarConfig _config; protected final String _clientId; - protected PulsarClient _pulsarClient = null; - protected final PulsarMetadataExtractor _pulsarMetadataExtractor; + protected final PulsarClient _pulsarClient; /** * Creates a new instance of {@link PulsarClient} and {@link Reader} */ - public PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig) { + protected PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig) { _config = new PulsarConfig(streamConfig, clientId); _clientId = clientId; - _pulsarMetadataExtractor = PulsarMetadataExtractor.build(_config.isPopulateMetadata(), _config.getMetadataFields()); try { ClientBuilder pulsarClientBuilder = PulsarClient.builder().serviceUrl(_config.getBootstrapServers()); - if (_config.getTlsTrustCertsFilePath() != null) { - pulsarClientBuilder.tlsTrustCertsFilePath(_config.getTlsTrustCertsFilePath()); + String tlsTrustCertsFilePath = _config.getTlsTrustCertsFilePath(); + if (StringUtils.isNotBlank(tlsTrustCertsFilePath)) { + pulsarClientBuilder.tlsTrustCertsFilePath(tlsTrustCertsFilePath); } - - if (_config.getAuthenticationToken() != null) { - Authentication authentication = AuthenticationFactory.token(_config.getAuthenticationToken()); - pulsarClientBuilder.authentication(authentication); + String authenticationToken = _config.getAuthenticationToken(); + if (StringUtils.isNotBlank(authenticationToken)) { + pulsarClientBuilder.authentication(AuthenticationFactory.token(authenticationToken)); + } else { + String issuerUrl = _config.getIssuerUrl(); + String credentialsFilePath = _config.getCredentialsFilePath(); + String audience = _config.getAudience(); + if (StringUtils.isNotBlank(issuerUrl) && StringUtils.isNotBlank(credentialsFilePath) && StringUtils.isNotBlank( + audience)) { + pulsarClientBuilder.authentication( + AuthenticationFactoryOAuth2.clientCredentials(new URL(issuerUrl), new URL(credentialsFilePath), + audience)); + } } - - getAuthenticationFactory(_config).ifPresent(pulsarClientBuilder::authentication); _pulsarClient = pulsarClientBuilder.build(); - LOGGER.info("Created pulsar client {}", _pulsarClient); } catch (Exception e) { - LOGGER.error("Could not create pulsar consumer", e); + throw new RuntimeException("Caught exception while creating Pulsar client", e); } } - protected Optional getAuthenticationFactory(PulsarConfig pulsarConfig) { - if (StringUtils.isNotBlank(pulsarConfig.getIssuerUrl()) - && StringUtils.isNotBlank(pulsarConfig.getAudience()) - && StringUtils.isNotBlank(pulsarConfig.getCredentialsFilePath())) { - try { - return Optional.of(AuthenticationFactoryOAuth2.clientCredentials( - new URL(pulsarConfig.getIssuerUrl()), - new URL(pulsarConfig.getCredentialsFilePath()), - pulsarConfig.getAudience())); - } catch (MalformedURLException mue) { - LOGGER.error("Failed to create authentication factory for pulsar client with config: " - + "issuer: {}, credential file path: {}, audience: {}", - pulsarConfig.getIssuerUrl(), - pulsarConfig.getCredentialsFilePath(), - pulsarConfig.getAudience(), - mue); - } - } - return Optional.empty(); - } - protected Reader createReaderForPartition(String topic, int partition, MessageId initialMessageId) { - if (_pulsarClient == null) { - throw new RuntimeException("Failed to create reader as no pulsar client found for topic " + topic); - } - try { - return _pulsarClient.newReader().topic(getPartitionedTopicName(topic, partition)).startMessageId(initialMessageId) - .startMessageIdInclusive().create(); - } catch (Exception e) { - LOGGER.error("Failed to create pulsar consumer client for topic " + topic + " partition " + partition, e); - return null; - } - } - - /** - * A pulsar partitioned topic with N partitions is comprised of N topics with topicName as prefix and portitionId - * as suffix. - * The method fetches the names of N partitioned topic and returns the topic name of {@param partition} - */ - protected String getPartitionedTopicName(String topic, int partition) - throws Exception { - List partitionTopicList = _pulsarClient.getPartitionsForTopic(topic).get(); - return partitionTopicList.get(partition); - } - + @Override public void close() throws IOException { - if (_pulsarClient != null) { - _pulsarClient.close(); - } + _pulsarClient.close(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java index d1b80b0360c8..c206574bc924 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java @@ -18,20 +18,15 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import com.google.common.collect.Iterables; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import java.util.Objects; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumer; -import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; @@ -45,92 +40,60 @@ public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class); - private final ExecutorService _executorService; - private final Reader _reader; - private boolean _enableKeyValueStitch; + private final Reader _reader; + private MessageId _nextMessageId = null; - public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, - PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, int partition) { super(clientId, streamConfig); - PulsarConfig config = new PulsarConfig(streamConfig, clientId); - _reader = - createReaderForPartition(config.getPulsarTopicName(), partitionGroupConsumptionStatus.getPartitionGroupId(), - config.getInitialMessageId()); - LOGGER.info("Created pulsar reader with id {} for topic {} partition {}", _reader, _config.getPulsarTopicName(), - partitionGroupConsumptionStatus.getPartitionGroupId()); - _executorService = Executors.newSingleThreadExecutor(); - _enableKeyValueStitch = _config.getEnableKeyValueStitch(); - } - - /** - * Fetch records from the Pulsar stream between the start and end StreamPartitionMsgOffset - * Used {@link org.apache.pulsar.client.api.Reader} to read the messaged from pulsar partitioned topic - * The reader seeks to the startMsgOffset and starts reading records in a loop until endMsgOffset or timeout is - * reached. - */ - @Override - public PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { - final MessageId startMessageId = ((MessageIdStreamOffset) startMsgOffset).getMessageId(); - final MessageId endMessageId = - endMsgOffset == null ? MessageId.latest : ((MessageIdStreamOffset) endMsgOffset).getMessageId(); - - List messagesList = new ArrayList<>(); - Future pulsarResultFuture = - _executorService.submit(() -> fetchMessages(startMessageId, endMessageId, messagesList)); - + String topicName = _config.getPulsarTopicName(); try { - return pulsarResultFuture.get(timeoutMillis, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - // The fetchMessages has thrown an exception. Most common cause is the timeout. - // We return the records fetched till now along with the next start offset. - pulsarResultFuture.cancel(true); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); + List partitions = _pulsarClient.getPartitionsForTopic(topicName).get(); + _reader = _pulsarClient.newReader().topic(partitions.get(partition)).startMessageId(MessageId.earliest) + .startMessageIdInclusive().create(); } catch (Exception e) { - LOGGER.warn("Error while fetching records from Pulsar", e); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); + throw new RuntimeException( + String.format("Caught exception while creating Pulsar reader for topic: %s, partition: %d", topicName, + partition), e); } + LOGGER.info("Created Pulsar reader for topic: {}, partition: {}", topicName, partition); } - public PulsarMessageBatch fetchMessages(MessageId startMessageId, MessageId endMessageId, - List messagesList) { - try { - _reader.seek(startMessageId); - - while (_reader.hasMessageAvailable()) { - Message nextMessage = _reader.readNext(); - - if (endMessageId != null) { - if (nextMessage.getMessageId().compareTo(endMessageId) > 0) { - break; - } - } - messagesList.add( - PulsarUtils.buildPulsarStreamMessage(nextMessage, _enableKeyValueStitch, _pulsarMetadataExtractor)); + @Override + public synchronized PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { + MessageId startMessageId = ((MessageIdStreamOffset) startOffset).getMessageId(); + long endTimeMs = System.currentTimeMillis() + timeoutMs; + List messages = new ArrayList<>(); - if (Thread.interrupted()) { - break; - } + // Seek to the start message id if necessary + // NOTE: Use Objects.equals() to check reference first for performance. + if (!Objects.equals(startMessageId, _nextMessageId)) { + try { + _reader.seek(startMessageId); + } catch (PulsarClientException e) { + throw new RuntimeException("Caught exception while seeking to message id: " + startMessageId, e); } + } - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); + // Read messages until all available messages are read, or we run out of time + try { + while (_reader.hasMessageAvailable() && System.currentTimeMillis() < endTimeMs) { + messages.add(PulsarUtils.buildPulsarStreamMessage(_reader.readNext(), _config)); + } } catch (PulsarClientException e) { - LOGGER.warn("Error consuming records from Pulsar topic", e); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); + throw new RuntimeException("Caught exception while fetching messages from Pulsar", e); } - } - private Iterable buildOffsetFilteringIterable(final List messageAndOffsets, - final MessageId startOffset, final MessageId endOffset) { - return Iterables.filter(messageAndOffsets, input -> { - // Filter messages that are either null or have an offset ∉ [startOffset, endOffset] - return input != null && input.getValue() != null && (input.getMessageId().compareTo(startOffset) >= 0) && ( - (endOffset == null) || (input.getMessageId().compareTo(endOffset) < 0)); - }); + MessageIdStreamOffset offsetOfNextBatch; + if (messages.isEmpty()) { + offsetOfNextBatch = (MessageIdStreamOffset) startOffset; + } else { + StreamMessageMetadata lastMessageMetadata = messages.get(messages.size() - 1).getMetadata(); + assert lastMessageMetadata != null; + offsetOfNextBatch = (MessageIdStreamOffset) lastMessageMetadata.getNextOffset(); + } + assert offsetOfNextBatch != null; + _nextMessageId = offsetOfNextBatch.getMessageId(); + return new PulsarMessageBatch(messages, offsetOfNextBatch, _reader.hasReachedEndOfTopic()); } @Override @@ -138,18 +101,5 @@ public void close() throws IOException { _reader.close(); super.close(); - shutdownAndAwaitTermination(); - } - - void shutdownAndAwaitTermination() { - _executorService.shutdown(); - try { - if (!_executorService.awaitTermination(60, TimeUnit.SECONDS)) { - _executorService.shutdownNow(); - } - } catch (InterruptedException ie) { - _executorService.shutdownNow(); - Thread.currentThread().interrupt(); - } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java deleted file mode 100644 index 7e0919785794..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * 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.pinot.plugin.stream.pulsar; - -import javax.annotation.Nullable; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pulsar.client.api.MessageId; - -public class PulsarStreamMessage extends StreamMessage { - - private final MessageId _messageId; - public PulsarStreamMessage(@Nullable byte[] key, byte[] value, MessageId messageId, - @Nullable PulsarStreamMessageMetadata metadata, int length) { - super(key, value, metadata, length); - _messageId = messageId; - } - - public MessageId getMessageId() { - return _messageId; - } - - int getKeyLength() { - byte[] key = getKey(); - return key == null ? 0 : key.length; - } - - int getValueLength() { - byte[] value = getValue(); - return value == null ? 0 : value.length; - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java index 59220138d73b..fcf219e98df5 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java @@ -25,52 +25,55 @@ import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.stream.StreamMessageMetadata; + /** * Pulsar specific implementation of {@link StreamMessageMetadata} * Pulsar makes many metadata values available for each message. Please see the pulsar documentation for more details. * @see Pulsar Message Properties */ +// TODO: Make it a util class public class PulsarStreamMessageMetadata extends StreamMessageMetadata { - public enum PulsarMessageMetadataValue { - PUBLISH_TIME("publishTime"), - EVENT_TIME("eventTime"), - BROKER_PUBLISH_TIME("brokerPublishTime"), - MESSAGE_KEY("key"), - MESSAGE_ID("messageId"), - MESSAGE_ID_BYTES_B64("messageIdBytes"), - PRODUCER_NAME("producerName"), - SCHEMA_VERSION("schemaVersion"), - SEQUENCE_ID("sequenceId"), - ORDERING_KEY("orderingKey"), - SIZE("size"), - TOPIC_NAME("topicName"), - INDEX("index"), - REDELIVERY_COUNT("redeliveryCount"); - - private final String _key; - - PulsarMessageMetadataValue(String key) { - _key = key; - } + public enum PulsarMessageMetadataValue { + PUBLISH_TIME("publishTime"), + EVENT_TIME("eventTime"), + BROKER_PUBLISH_TIME("brokerPublishTime"), + MESSAGE_KEY("key"), + MESSAGE_ID("messageId"), + MESSAGE_ID_BYTES_B64("messageIdBytes"), + PRODUCER_NAME("producerName"), + SCHEMA_VERSION("schemaVersion"), + SEQUENCE_ID("sequenceId"), + ORDERING_KEY("orderingKey"), + SIZE("size"), + TOPIC_NAME("topicName"), + INDEX("index"), + REDELIVERY_COUNT("redeliveryCount"); - public String getKey() { - return _key; - } + private final String _key; - public static PulsarMessageMetadataValue findByKey(final String key) { - EnumSet values = EnumSet.allOf(PulsarMessageMetadataValue.class); - return values.stream().filter(value -> value.getKey().equals(key)).findFirst().orElse(null); - } + PulsarMessageMetadataValue(String key) { + _key = key; } - public PulsarStreamMessageMetadata(long recordIngestionTimeMs, - @Nullable GenericRow headers) { - super(recordIngestionTimeMs, headers); + public String getKey() { + return _key; } - public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, - Map metadata) { - super(recordIngestionTimeMs, headers, metadata); + public static PulsarMessageMetadataValue findByKey(final String key) { + EnumSet values = EnumSet.allOf(PulsarMessageMetadataValue.class); + return values.stream().filter(value -> value.getKey().equals(key)).findFirst().orElse(null); } + } + + @Deprecated + public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers) { + super(recordIngestionTimeMs, headers); + } + + @Deprecated + public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, + Map metadata) { + super(recordIngestionTimeMs, headers, metadata); + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java index 772357aa6baf..0ccacc304704 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java @@ -18,11 +18,25 @@ */ package org.apache.pinot.plugin.stream.pulsar; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; import java.nio.ByteBuffer; +import java.util.Base64; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.OffsetCriteria; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.BatchMessageAcker; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; public class PulsarUtils { @@ -39,16 +53,6 @@ public static SubscriptionInitialPosition offsetCriteriaToSubscription(OffsetCri throw new IllegalArgumentException("Unsupported offset criteria: " + offsetCriteria); } - public static MessageId offsetCriteriaToMessageId(OffsetCriteria offsetCriteria) { - if (offsetCriteria.isLargest()) { - return MessageId.latest; - } - if (offsetCriteria.isSmallest()) { - return MessageId.earliest; - } - throw new IllegalArgumentException("Unsupported offset criteria: " + offsetCriteria); - } - /** * Stitch key and value bytes together using a simple format: * 4 bytes for key length + key bytes + 4 bytes for value length + value bytes @@ -63,14 +67,170 @@ public static byte[] stitchKeyValue(byte[] keyBytes, byte[] valueBytes) { return stitchedBytes; } - public static PulsarStreamMessage buildPulsarStreamMessage(Message message, boolean enableKeyValueStitch, - PulsarMetadataExtractor pulsarMetadataExtractor) { + public static BytesStreamMessage buildPulsarStreamMessage(Message message, PulsarConfig config) { byte[] key = message.getKeyBytes(); byte[] value = message.getData(); - if (enableKeyValueStitch) { + if (config.getEnableKeyValueStitch()) { value = stitchKeyValue(key, value); } - return new PulsarStreamMessage(key, value, message.getMessageId(), - (PulsarStreamMessageMetadata) pulsarMetadataExtractor.extract(message), value.length); + return new BytesStreamMessage(key, value, extractMessageMetadata(message, config)); + } + + @VisibleForTesting + static StreamMessageMetadata extractMessageMetadata(Message message, PulsarConfig config) { + long recordIngestionTimeMs = message.getBrokerPublishTime().orElse(message.getPublishTime()); + MessageId messageId = message.getMessageId(); + MessageIdStreamOffset offset = new MessageIdStreamOffset(messageId); + MessageIdStreamOffset nextOffset = new MessageIdStreamOffset(getNextMessageId(messageId)); + StreamMessageMetadata.Builder builder = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(recordIngestionTimeMs) + .setOffset(offset, nextOffset); + if (config.isPopulateMetadata()) { + Map properties = message.getProperties(); + if (!properties.isEmpty()) { + GenericRow header = new GenericRow(); + for (Map.Entry entry : properties.entrySet()) { + header.putValue(entry.getKey(), entry.getValue()); + } + builder.setHeaders(header); + } + Set metadataFields = config.getMetadataFields(); + if (!metadataFields.isEmpty()) { + Map metadataMap = Maps.newHashMapWithExpectedSize(metadataFields.size()); + for (PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataField : metadataFields) { + populateMetadataField(message, metadataField, metadataMap); + } + builder.setMetadata(metadataMap); + } + } + return builder.build(); + } + + /** + * Returns next message id supposed to be present in the pulsar topic partition. + * + * The message id is composed of 3 parts - ledgerId, entryId and partitionId. + * The ledger id are always increasing in number but may not be sequential. e.g. for first 10 records ledger id can + * be 12 but for next 10 it can be 18. Each entry inside a ledger is always in a sequential and increases by 1 for + * next message. + * The partition id is fixed for a particular partition. + * We return entryId incremented by 1 while keeping ledgerId and partitionId as same. + * If ledgerId has incremented, the {@link Reader} takes care of that during seek operation, and returns the first + * record in the new ledger. + */ + public static MessageId getNextMessageId(MessageId messageId) { + MessageIdImpl messageIdImpl = MessageIdImpl.convertToMessageIdImpl(messageId); + long ledgerId = messageIdImpl.getLedgerId(); + long entryId = messageIdImpl.getEntryId(); + int partitionIndex = messageIdImpl.getPartitionIndex(); + if (messageIdImpl instanceof BatchMessageIdImpl) { + BatchMessageIdImpl batchMessageIdImpl = (BatchMessageIdImpl) messageIdImpl; + int batchIndex = batchMessageIdImpl.getBatchIndex(); + int batchSize = batchMessageIdImpl.getBatchSize(); + BatchMessageAcker acker = batchMessageIdImpl.getAcker(); + if (batchIndex < batchSize - 1) { + return new BatchMessageIdImpl(ledgerId, entryId, partitionIndex, batchIndex + 1, batchSize, acker); + } else { + return new BatchMessageIdImpl(ledgerId, entryId + 1, partitionIndex, 0, batchSize, acker); + } + } else { + return DefaultImplementation.getDefaultImplementation().newMessageId(ledgerId, entryId + 1, partitionIndex); + } + } + + private static void populateMetadataField(Message message, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataField, Map metadataMap) { + switch (metadataField) { + case PUBLISH_TIME: + long publishTime = message.getPublishTime(); + if (publishTime > 0) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, + publishTime); + } + break; + case EVENT_TIME: + long eventTime = message.getEventTime(); + if (eventTime > 0) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, + eventTime); + } + break; + case BROKER_PUBLISH_TIME: + message.getBrokerPublishTime().ifPresent(brokerPublishTime -> setMetadataMapField(metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, brokerPublishTime)); + break; + case MESSAGE_KEY: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, + message.getKey()); + break; + case MESSAGE_ID: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, + message.getMessageId().toString()); + break; + case MESSAGE_ID_BYTES_B64: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, + message.getMessageId().toByteArray()); + break; + case PRODUCER_NAME: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PRODUCER_NAME, + message.getProducerName()); + break; + case SCHEMA_VERSION: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SCHEMA_VERSION, + message.getSchemaVersion()); + break; + case SEQUENCE_ID: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SEQUENCE_ID, + message.getSequenceId()); + break; + case ORDERING_KEY: + if (message.hasOrderingKey()) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.ORDERING_KEY, + message.getOrderingKey()); + } + break; + case SIZE: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SIZE, message.size()); + break; + case TOPIC_NAME: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME, + message.getTopicName()); + break; + case INDEX: + message.getIndex().ifPresent( + index -> setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.INDEX, + index)); + break; + case REDELIVERY_COUNT: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.REDELIVERY_COUNT, + message.getRedeliveryCount()); + break; + default: + throw new IllegalArgumentException("Unsupported metadata field: " + metadataField); + } + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, String value) { + if (StringUtils.isNotEmpty(value)) { + metadataMap.put(metadataValue.getKey(), value); + } + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, int value) { + setMetadataMapField(metadataMap, metadataValue, Integer.toString(value)); + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, long value) { + setMetadataMapField(metadataMap, metadataValue, Long.toString(value)); + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, byte[] value) { + if (value != null && value.length > 0) { + setMetadataMapField(metadataMap, metadataValue, Base64.getEncoder().encodeToString(value)); + } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java index 486944b70384..bca1416cecfb 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java @@ -31,14 +31,13 @@ import org.testng.Assert; import org.testng.annotations.Test; -import static org.junit.Assert.assertTrue; public class PulsarConfigTest { public static final String TABLE_NAME_WITH_TYPE = "tableName_REALTIME"; - public static final String STREAM_TYPE = "pulsar"; public static final String STREAM_PULSAR_BROKER_LIST = "pulsar://localhost:6650"; public static final String STREAM_PULSAR_CONSUMER_TYPE = "simple"; + Map getCommonStreamConfigMap() { Map streamConfigMap = new HashMap<>(); streamConfigMap.put("streamType", STREAM_TYPE); @@ -55,7 +54,7 @@ Map getCommonStreamConfigMap() { } @Test - public void testParsingMetadataConfigWithConfiguredFields() throws Exception { + public void testParsingMetadataConfigWithConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "true"); @@ -66,21 +65,20 @@ public void testParsingMetadataConfigWithConfiguredFields() throws Exception { Set metadataFieldsToExtract = pulsarConfig.getMetadataFields(); Assert.assertEquals(metadataFieldsToExtract.size(), 6); - Assert.assertTrue(metadataFieldsToExtract.containsAll(ImmutableList.of( - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME))); + Assert.assertTrue(metadataFieldsToExtract.containsAll( + ImmutableList.of(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME))); } @Test - public void testParsingMetadataConfigWithoutConfiguredFields() throws Exception { + public void testParsingMetadataConfigWithoutConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "true"); + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "true"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Set metadataFieldsToExtract = @@ -89,11 +87,10 @@ public void testParsingMetadataConfigWithoutConfiguredFields() throws Exception } @Test - public void testParsingNoMetadataConfig() throws Exception { + public void testParsingNoMetadataConfig() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "false"); + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "false"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Assert.assertFalse(pulsarConfig.isPopulateMetadata()); @@ -103,13 +100,11 @@ public void testParsingNoMetadataConfig() throws Exception { } @Test - public void testParsingNoMetadataConfigWithConfiguredFields() throws Exception { + public void testParsingNoMetadataConfigWithConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "false"); - streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.METADATA_FIELDS), + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "false"); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.METADATA_FIELDS), "messageId,messageIdBytes, publishTime, eventTime, key, topicName, "); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); @@ -120,55 +115,42 @@ public void testParsingNoMetadataConfigWithConfiguredFields() throws Exception { } @Test - public void testParsingConfigForOAuth() throws Exception { + public void testParsingConfigForOAuth() + throws Exception { Path testFile = null; try { testFile = Files.createTempFile("test_cred_file", ".json"); Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), "http://auth.test.com"); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, - PulsarConfig.OAUTH_CREDS_FILE_PATH), "file://" + testFile.toFile().getAbsolutePath()); + streamConfigMap.put( + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_CREDS_FILE_PATH), + "file://" + testFile.toFile().getAbsolutePath()); streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), "urn:test:test"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Assert.assertEquals(pulsarConfig.getIssuerUrl(), "http://auth.test.com"); - Assert.assertEquals(pulsarConfig.getCredentialsFilePath(), - "file://" + testFile.toFile().getAbsolutePath()); + Assert.assertEquals(pulsarConfig.getCredentialsFilePath(), "file://" + testFile.toFile().getAbsolutePath()); Assert.assertEquals(pulsarConfig.getAudience(), "urn:test:test"); - PulsarPartitionLevelConnectionHandler pulsarPartitionLevelConnectionHandler = - new PulsarPartitionLevelConnectionHandler("testId", streamConfig); - assertTrue(pulsarPartitionLevelConnectionHandler.getAuthenticationFactory(pulsarConfig).isPresent()); - } catch (Exception e) { - Assert.fail("Should not throw exception", e); } finally { Optional.ofNullable(testFile).map(Path::toFile).ifPresent(File::delete); } } - @Test - public void testParsingConfigFailFileValidationForOAuth() throws Exception { + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid credentials " + + "file path: .*") + public void testParsingConfigFailFileValidationForOAuth() { String testFilePath = "file://path/to/file.json"; - - try { - Map streamConfigMap = getCommonStreamConfigMap(); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), - "http://auth.test.com"); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, - PulsarConfig.OAUTH_CREDS_FILE_PATH), - testFilePath); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), - "urn:test:test"); - StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); - PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); //will throw exception - } catch (IllegalArgumentException mue) { - //expected case. - String errorMessage = String.format("Invalid credentials file path: %s. File does not exist.", testFilePath); - Assert.assertEquals(errorMessage, mue.getMessage()); - } catch (Exception e) { - Assert.fail("Should not throw other exception", e); - } + Map streamConfigMap = getCommonStreamConfigMap(); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), + "http://auth.test.com"); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_CREDS_FILE_PATH), + testFilePath); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), + "urn:test:test"); + StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); + new PulsarConfig(streamConfig, "testId"); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java index 9d59f82fcc8e..01cd5cd26e0f 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java @@ -18,24 +18,18 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import com.google.common.base.Function; import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import javax.annotation.Nullable; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Message; @@ -45,72 +39,54 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TopicMetadata; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.PulsarContainer; import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; -public class PulsarConsumerTest { - private static final Logger LOGGER = LoggerFactory.getLogger(PulsarConsumerTest.class); - private static final DockerImageName PULSAR_IMAGE = DockerImageName.parse("apachepulsar/pulsar:2.7.2"); +public class PulsarConsumerTest { + private static final DockerImageName PULSAR_IMAGE = DockerImageName.parse("apachepulsar/pulsar:2.11.4"); public static final String TABLE_NAME_WITH_TYPE = "tableName_REALTIME"; public static final String TEST_TOPIC = "test-topic"; public static final String TEST_TOPIC_BATCH = "test-topic-batch"; - public static final String MESSAGE_PREFIX = "sample_msg"; + public static final String MESSAGE_PREFIX = "sample_msg_"; public static final String CLIENT_ID = "clientId"; - public static final int NUM_PARTITION = 1; + public static final int NUM_PARTITIONS = 2; public static final int NUM_RECORDS_PER_PARTITION = 1000; public static final int BATCH_SIZE = 10; - public static final int CONSUMER_FETCH_TIMEOUT_MILLIS = (int) Duration.ofMinutes(5).toMillis(); + public static final int CONSUMER_FETCH_TIMEOUT_MILLIS = (int) TimeUnit.MINUTES.toMillis(1); + private final List> _partitionToMessageIdMapping = new ArrayList<>(NUM_PARTITIONS); + private final List> _partitionToMessageIdMappingBatch = new ArrayList<>(NUM_PARTITIONS); + + private PulsarContainer _pulsar; private PulsarClient _pulsarClient; - private PulsarContainer _pulsar = null; - private HashMap _partitionToFirstMessageIdMap = new HashMap<>(); - private HashMap _partitionToFirstMessageIdMapBatch = new HashMap<>(); - private ConcurrentHashMap> _partitionToMessageIdMapping = new ConcurrentHashMap<>(); @BeforeClass public void setUp() throws Exception { + _pulsar = new PulsarContainer(PULSAR_IMAGE).withStartupTimeout(Duration.ofMinutes(5)); try { - _pulsar = new PulsarContainer(PULSAR_IMAGE).withStartupTimeout(Duration.ofMinutes(5)); _pulsar.start(); - - // Waiting for namespace to be created. - // There should be a better approach. - Thread.sleep(20 * 1000L); - - PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(_pulsar.getHttpServiceUrl()).build(); - - String bootstrapServer = _pulsar.getPulsarBrokerUrl(); - - _pulsarClient = PulsarClient.builder().serviceUrl(bootstrapServer).build(); - - createTopics(admin); - - publishRecords(); - publishRecordsBatch(); - - waitForMessagesToPublish(admin, TEST_TOPIC); - waitForMessagesToPublish(admin, TEST_TOPIC_BATCH); - - admin.close(); - } catch (Exception e) { - if (_pulsar != null) { - _pulsar.stop(); - _pulsar = null; + _pulsarClient = PulsarClient.builder().serviceUrl(_pulsar.getPulsarBrokerUrl()).build(); + + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(_pulsar.getHttpServiceUrl()).build()) { + createTopics(admin); + publishRecords(); + publishRecordsBatch(); + waitForMessagesToPublish(admin, TEST_TOPIC); + waitForMessagesToPublish(admin, TEST_TOPIC_BATCH); } + } catch (Exception e) { + _pulsar.stop(); throw new RuntimeException("Failed to setUp test environment", e); } } @@ -121,50 +97,36 @@ private void createTopics(PulsarAdmin admin) inactiveTopicPolicies.setDeleteWhileInactive(false); admin.namespaces().setInactiveTopicPolicies("public/default", inactiveTopicPolicies); - admin.topics().createPartitionedTopic(TEST_TOPIC, NUM_PARTITION); - admin.topics().createPartitionedTopic(TEST_TOPIC_BATCH, NUM_PARTITION); + admin.topics().createPartitionedTopic(TEST_TOPIC, NUM_PARTITIONS); + admin.topics().createPartitionedTopic(TEST_TOPIC_BATCH, NUM_PARTITIONS); } - private void waitForMessagesToPublish(PulsarAdmin admin, String topicName) { - waitForCondition(new Function() { - @Nullable - @Override - public Boolean apply(@Nullable Void aVoid) { - try { - return getNumberOfEntries(admin, topicName) == NUM_RECORDS_PER_PARTITION * NUM_PARTITION; - } catch (Exception e) { - LOGGER.warn("Could not fetch number of messages in pulsar topic " + topicName, e); - return null; - } + private void waitForMessagesToPublish(PulsarAdmin admin, String topicName) + throws Exception { + long endTimeMs = System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(5); + while (System.currentTimeMillis() < endTimeMs) { + if (admin.topics().getPartitionedStats(topicName, false).getMsgInCounter() + == NUM_RECORDS_PER_PARTITION * NUM_PARTITIONS) { + return; } - }, 2000L, 60 * 1000L, "Failed to produce " + NUM_RECORDS_PER_PARTITION * NUM_PARTITION + " messages", true); - } - - private long getNumberOfEntries(PulsarAdmin admin, String topicName) { - try { - return admin.topics().getPartitionedStats(topicName, false).getMsgInCounter(); - } catch (Exception e) { - e.printStackTrace(); - LOGGER.warn("Could not fetch number of rows in pulsar topic " + topicName, e); + Thread.sleep(1000); } - return -1; + throw new RuntimeException("Failed to publish messages to topic: " + topicName); } @AfterClass public void tearDown() throws Exception { - if (_pulsar != null) { - _pulsar.stop(); - _pulsarClient.close(); - _partitionToMessageIdMapping.clear(); - _pulsar = null; - } + _pulsarClient.close(); + _pulsar.stop(); } public void publishRecords() throws Exception { - for (int p = 0; p < NUM_PARTITION; p++) { - final int partition = p; + for (int p = 0; p < NUM_PARTITIONS; p++) { + List messageIds = new ArrayList<>(NUM_RECORDS_PER_PARTITION); + _partitionToMessageIdMapping.add(messageIds); + int partition = p; try (Producer producer = _pulsarClient.newProducer(Schema.STRING).topic(TEST_TOPIC) .messageRouter(new MessageRouter() { @Override @@ -173,23 +135,19 @@ public int choosePartition(Message msg, TopicMetadata metadata) { } }).create()) { for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) { - MessageId messageId = producer.send(MESSAGE_PREFIX + "_" + i); - if (!_partitionToFirstMessageIdMap.containsKey(partition)) { - _partitionToFirstMessageIdMap.put(partition, messageId); - } + messageIds.add(producer.send(MESSAGE_PREFIX + i)); } producer.flush(); } - waitForCondition(input -> validatePartitionMessageCount(partition, NUM_RECORDS_PER_PARTITION, TEST_TOPIC), - 1 * 1000L, 5 * 60 * 1000L, - "Failed to consume " + NUM_RECORDS_PER_PARTITION + " messages from partition " + partition, true); } } public void publishRecordsBatch() throws Exception { - for (int p = 0; p < NUM_PARTITION; p++) { - final int partition = p; + for (int p = 0; p < NUM_PARTITIONS; p++) { + List messageIds = new ArrayList<>(NUM_RECORDS_PER_PARTITION); + _partitionToMessageIdMappingBatch.add(messageIds); + int partition = p; try (Producer producer = _pulsarClient.newProducer(Schema.STRING).topic(TEST_TOPIC_BATCH) .messageRouter(new MessageRouter() { @Override @@ -198,207 +156,115 @@ public int choosePartition(Message msg, TopicMetadata metadata) { } }).batchingMaxMessages(BATCH_SIZE).batchingMaxPublishDelay(1, TimeUnit.SECONDS).create()) { for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) { - CompletableFuture messageIdCompletableFuture = producer.sendAsync(MESSAGE_PREFIX + "_" + i); - messageIdCompletableFuture.thenAccept(messageId -> { - - _partitionToMessageIdMapping.compute(partition, (partitionId, messageIds) -> { - if (messageIds == null) { - List messageIdList = new ArrayList<>(); - messageIdList.add((BatchMessageIdImpl) messageId); - if (!_partitionToFirstMessageIdMapBatch.containsKey(partition)) { - _partitionToFirstMessageIdMapBatch.put(partition, messageId); - } - return messageIdList; - } else { - messageIds.add((BatchMessageIdImpl) messageId); - return messageIds; - } - }); - }); + messageIds.add(producer.send(MESSAGE_PREFIX + i)); } producer.flush(); } - waitForCondition(input -> validatePartitionMessageCount(partition, NUM_RECORDS_PER_PARTITION, TEST_TOPIC_BATCH), - 1 * 1000L, 5 * 60 * 1000L, - "Failed to consume " + NUM_RECORDS_PER_PARTITION + " messages from partition " + partition, true); - } - } - - private boolean validatePartitionMessageCount(int partition, int expectedMsgCount, String topicName) { - final PartitionGroupConsumer consumer = StreamConsumerFactoryProvider.create(getStreamConfig(topicName)) - .createPartitionGroupConsumer(CLIENT_ID, - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, - "CONSUMING")); - try { - final MessageBatch messageBatch = - consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), null, CONSUMER_FETCH_TIMEOUT_MILLIS); - LOGGER.info("Partition: " + partition + ", Consumed messageBatch count = " + messageBatch.getMessageCount()); - return messageBatch.getMessageCount() == expectedMsgCount; - } catch (TimeoutException e) { - return false; } } public StreamConfig getStreamConfig(String topicName) { - String streamType = "pulsar"; - String streamPulsarBrokerList = _pulsar.getPulsarBrokerUrl(); - String streamPulsarConsumerType = "simple"; - String tableNameWithType = TABLE_NAME_WITH_TYPE; - Map streamConfigMap = new HashMap<>(); - streamConfigMap.put("streamType", streamType); - streamConfigMap.put("stream.pulsar.consumer.type", streamPulsarConsumerType); + streamConfigMap.put("streamType", "pulsar"); + streamConfigMap.put("stream.pulsar.consumer.type", "simple"); streamConfigMap.put("stream.pulsar.topic.name", topicName); - streamConfigMap.put("stream.pulsar.bootstrap.servers", streamPulsarBrokerList); + streamConfigMap.put("stream.pulsar.bootstrap.servers", _pulsar.getPulsarBrokerUrl()); streamConfigMap.put("stream.pulsar.consumer.prop.auto.offset.reset", "smallest"); - streamConfigMap.put("stream.pulsar.consumer.factory.class.name", getPulsarConsumerFactoryName()); - streamConfigMap.put( - StreamConfigProperties.constructStreamProperty("pulsar", StreamConfigProperties.STREAM_FETCH_TIMEOUT_MILLIS), - "1000"); - streamConfigMap.put("stream.pulsar.decoder.class.name", "decoderClass"); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - - return streamConfig; - } - - protected String getPulsarConsumerFactoryName() { - return PulsarConsumerFactory.class.getName(); + streamConfigMap.put("stream.pulsar.consumer.factory.class.name", PulsarConsumerFactory.class.getName()); + streamConfigMap.put("stream.pulsar.decoder.class.name", "dummy"); + return new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); } @Test public void testPartitionLevelConsumer() throws Exception { - - final StreamConsumerFactory streamConsumerFactory = - StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC)); - int numPartitions = new PulsarStreamMetadataProvider(CLIENT_ID, getStreamConfig(TEST_TOPIC)).fetchPartitionCount( - CONSUMER_FETCH_TIMEOUT_MILLIS); + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC)); + int numPartitions; + try (PulsarStreamMetadataProvider metadataProvider = new PulsarStreamMetadataProvider(CLIENT_ID, + getStreamConfig(TEST_TOPIC))) { + numPartitions = metadataProvider.fetchPartitionCount(CONSUMER_FETCH_TIMEOUT_MILLIS); + } for (int partition = 0; partition < numPartitions; partition++) { PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, + new PartitionGroupConsumptionStatus(partition, 0, new MessageIdStreamOffset(MessageId.earliest), null, "CONSUMING"); + try ( + PulsarPartitionLevelConsumer consumer = + (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer( + CLIENT_ID, partitionGroupConsumptionStatus)) { + PulsarMessageBatch messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 1000); + assertFalse(messageBatch.isEndOfPartitionGroup()); + for (int i = 0; i < 1000; i++) { + verifyMessage(messageBatch.getStreamMessage(i), partition, i, false); + } - int totalMessagesReceived = 0; - - final PulsarPartitionLevelConsumer consumer = - (PulsarPartitionLevelConsumer) streamConsumerFactory - .createPartitionGroupConsumer(CLIENT_ID, partitionGroupConsumptionStatus); - final PulsarMessageBatch messageBatch1 = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), - new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch1.getMessageCount(), 500); - for (int i = 0; i < messageBatch1.getMessageCount(); i++) { - final byte[] msg = messageBatch1.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch2 = - consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), null, - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch2.getMessageCount(), 500); - for (int i = 0; i < messageBatch2.getMessageCount(); i++) { - final byte[] msg = messageBatch2.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch3 = - consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 10)), - new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 35)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch3.getMessageCount(), 25); - for (int i = 0; i < messageBatch3.getMessageCount(); i++) { - final byte[] msg = messageBatch3.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); + messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(_partitionToMessageIdMapping.get(partition).get(500)), + CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 500); + assertFalse(messageBatch.isEndOfPartitionGroup()); + for (int i = 0; i < 500; i++) { + verifyMessage(messageBatch.getStreamMessage(i), partition, 500 + i, false); + } } + } + } - Assert.assertEquals(totalMessagesReceived, NUM_RECORDS_PER_PARTITION); + private void verifyMessage(BytesStreamMessage streamMessage, int partition, int index, boolean batch) { + assertEquals(new String(streamMessage.getValue()), MESSAGE_PREFIX + index); + StreamMessageMetadata messageMetadata = streamMessage.getMetadata(); + assertNotNull(messageMetadata); + MessageIdStreamOffset offset = (MessageIdStreamOffset) messageMetadata.getOffset(); + assertNotNull(offset); + MessageIdStreamOffset nextOffset = (MessageIdStreamOffset) messageMetadata.getNextOffset(); + assertNotNull(nextOffset); + List messageIds = + batch ? _partitionToMessageIdMappingBatch.get(partition) : _partitionToMessageIdMapping.get(partition); + assertEquals(offset.getMessageId(), messageIds.get(index)); + if (index < NUM_RECORDS_PER_PARTITION - 1) { + assertEquals(nextOffset.getMessageId(), messageIds.get(index + 1)); } } @Test public void testPartitionLevelConsumerBatchMessages() throws Exception { - - final StreamConsumerFactory streamConsumerFactory = + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC_BATCH)); - int numPartitions = - new PulsarStreamMetadataProvider(CLIENT_ID, getStreamConfig(TEST_TOPIC_BATCH)).fetchPartitionCount( - CONSUMER_FETCH_TIMEOUT_MILLIS); + int numPartitions; + try (PulsarStreamMetadataProvider metadataProvider = new PulsarStreamMetadataProvider(CLIENT_ID, + getStreamConfig(TEST_TOPIC_BATCH))) { + numPartitions = metadataProvider.fetchPartitionCount(CONSUMER_FETCH_TIMEOUT_MILLIS); + } for (int partition = 0; partition < numPartitions; partition++) { PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, + new PartitionGroupConsumptionStatus(partition, 0, new MessageIdStreamOffset(MessageId.earliest), null, "CONSUMING"); + try ( + PulsarPartitionLevelConsumer consumer = + (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer( + CLIENT_ID, partitionGroupConsumptionStatus)) { + PulsarMessageBatch messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 1000); + assertFalse(messageBatch.isEndOfPartitionGroup()); + for (int i = 0; i < 1000; i++) { + verifyMessage(messageBatch.getStreamMessage(i), partition, i, true); + } - int totalMessagesReceived = 0; - - final PulsarPartitionLevelConsumer consumer = - (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer(CLIENT_ID, - partitionGroupConsumptionStatus); - //TODO: This test failed, check it out. - final PulsarMessageBatch messageBatch1 = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), - new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch1.getMessageCount(), 500); - for (int i = 0; i < messageBatch1.getMessageCount(); i++) { - final byte[] msg = messageBatch1.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch2 = - consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), null, - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch2.getMessageCount(), 500); - for (int i = 0; i < messageBatch2.getMessageCount(); i++) { - final byte[] msg = messageBatch2.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch3 = - consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 10)), - new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 35)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch3.getMessageCount(), 25); - for (int i = 0; i < messageBatch3.getMessageCount(); i++) { - final byte[] msg = messageBatch3.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); - } - - Assert.assertEquals(totalMessagesReceived, NUM_RECORDS_PER_PARTITION); - } - } - - private MessageId getMessageIdForPartitionAndIndex(int partitionNum, int index) { - MessageId startMessageIdRaw = _partitionToFirstMessageIdMap.get(partitionNum); - MessageIdImpl startMessageId = MessageIdImpl.convertToMessageIdImpl(startMessageIdRaw); - return DefaultImplementation.getDefaultImplementation() - .newMessageId(startMessageId.getLedgerId(), index, partitionNum); - } - - private MessageId getBatchMessageIdForPartitionAndIndex(int partitionNum, int index) { - return _partitionToMessageIdMapping.get(partitionNum).get(index); - } - - private void waitForCondition(Function condition, long checkIntervalMs, long timeoutMs, - @Nullable String errorMessage, boolean raiseError) { - long endTime = System.currentTimeMillis() + timeoutMs; - String errorMessageSuffix = errorMessage != null ? ", error message: " + errorMessage : ""; - while (System.currentTimeMillis() < endTime) { - try { - if (Boolean.TRUE.equals(condition.apply(null))) { - return; + messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(_partitionToMessageIdMappingBatch.get(partition).get(500)), + CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 500); + assertFalse(messageBatch.isEndOfPartitionGroup()); + for (int i = 0; i < 500; i++) { + verifyMessage(messageBatch.getStreamMessage(i), partition, 500 + i, true); } - Thread.sleep(checkIntervalMs); - } catch (Exception e) { - Assert.fail("Caught exception while checking the condition" + errorMessageSuffix, e); } } - if (raiseError) { - Assert.fail("Failed to meet condition in " + timeoutMs + "ms" + errorMessageSuffix); - } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java index 904dd33a0461..f95dea1279ef 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java @@ -19,34 +19,35 @@ package org.apache.pinot.plugin.stream.pulsar; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; -import java.util.stream.Collectors; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.api.EncryptionContext; -import org.junit.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; + public class PulsarMessageBatchTest { - private Random _random = new Random(); - private DummyPulsarMessage _msgWithKeyAndValue; + private static final Random RANDOM = new Random(); + private byte[] _expectedValueBytes; private byte[] _expectedKeyBytes; - private List _messageList; - private PulsarMetadataExtractor _metadataExtractor; + private DummyPulsarMessage _message; public static class DummyPulsarMessage implements Message { private final byte[] _keyData; private final byte[] _valueData; - private Map _properties; + private final Map _properties; public DummyPulsarMessage(byte[] key, byte[] value) { _keyData = key; @@ -111,7 +112,7 @@ public String getProducerName() { @Override public boolean hasKey() { - return _keyData == null ? false : true; + return _keyData != null; } @Override @@ -198,45 +199,39 @@ public Optional getIndex() { public void setup() { _expectedValueBytes = new byte[10]; _expectedKeyBytes = new byte[10]; - _random.nextBytes(_expectedValueBytes); - _random.nextBytes(_expectedKeyBytes); - _msgWithKeyAndValue = new DummyPulsarMessage(_expectedKeyBytes, _expectedValueBytes); - _messageList = new ArrayList<>(); - _metadataExtractor = PulsarMetadataExtractor.build(true, - EnumSet.allOf(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.class)); - _messageList.add(_msgWithKeyAndValue); + RANDOM.nextBytes(_expectedValueBytes); + RANDOM.nextBytes(_expectedKeyBytes); + _message = new DummyPulsarMessage(_expectedKeyBytes, _expectedValueBytes); } @Test public void testMessageBatchNoStitching() { - List streamMessages = _messageList.stream().map(message -> - PulsarUtils.buildPulsarStreamMessage(message, false, _metadataExtractor)) - .collect(Collectors.toList()); - PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, false); - byte[] valueBytes = messageBatch.getMessageAtIndex(0).getValue(); - Assert.assertArrayEquals(_expectedValueBytes, valueBytes); + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(false); + List streamMessages = List.of(PulsarUtils.buildPulsarStreamMessage(_message, config)); + PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, mock(MessageIdStreamOffset.class), false); + byte[] valueBytes = messageBatch.getStreamMessage(0).getValue(); + assertEquals(valueBytes, _expectedValueBytes); } @Test public void testMessageBatchWithStitching() { - List streamMessages = _messageList.stream().map(message -> - PulsarUtils.buildPulsarStreamMessage(message, true, _metadataExtractor)) - .collect(Collectors.toList()); - PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, true); - byte[] keyValueBytes = messageBatch.getMessageAtIndex(0).getValue(); - Assert.assertEquals(keyValueBytes.length, 8 + _expectedKeyBytes.length + _expectedValueBytes.length); - try { - ByteBuffer byteBuffer = ByteBuffer.wrap(keyValueBytes); - int keyLength = byteBuffer.getInt(); - byte[] keyBytes = new byte[keyLength]; - byteBuffer.get(keyBytes); - Assert.assertArrayEquals(_expectedKeyBytes, keyBytes); - int valueLength = byteBuffer.getInt(); - byte[] valueBytes = new byte[valueLength]; - byteBuffer.get(valueBytes); - Assert.assertArrayEquals(_expectedValueBytes, valueBytes); - } catch (Exception e) { - Assert.fail("Could not parse key and value bytes because of exception: " + e.getMessage()); - } + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(true); + List streamMessages = List.of(PulsarUtils.buildPulsarStreamMessage(_message, config)); + PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, mock(MessageIdStreamOffset.class), false); + BytesStreamMessage streamMessage = messageBatch.getStreamMessage(0); + byte[] keyValueBytes = streamMessage.getValue(); + assertNotNull(keyValueBytes); + assertEquals(keyValueBytes.length, 8 + _expectedKeyBytes.length + _expectedValueBytes.length); + ByteBuffer byteBuffer = ByteBuffer.wrap(keyValueBytes); + int keyLength = byteBuffer.getInt(); + byte[] keyBytes = new byte[keyLength]; + byteBuffer.get(keyBytes); + assertEquals(keyBytes, _expectedKeyBytes); + int valueLength = byteBuffer.getInt(); + byte[] valueBytes = new byte[valueLength]; + byteBuffer.get(valueBytes); + assertEquals(valueBytes, _expectedValueBytes); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java similarity index 57% rename from pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java rename to pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java index fd4eb1fa6a61..be1c5050c8fa 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java @@ -18,76 +18,78 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import com.google.common.collect.ImmutableSet; import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Set; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.api.MessageId; import org.bouncycastle.util.encoders.Base64; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; import static org.apache.pinot.plugin.stream.pulsar.PulsarMessageBatchTest.DummyPulsarMessage; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; -public class PulsarMetadataExtractorTest { - - private PulsarMetadataExtractor _metadataExtractor; - - @BeforeClass - public void setup() { - _metadataExtractor = - PulsarMetadataExtractor.build(true, ImmutableSet.of(MESSAGE_ID, MESSAGE_ID_BYTES_B64, MESSAGE_KEY)); - } +public class PulsarUtilsTest { @Test public void testExtractProperty() throws Exception { + PulsarConfig config = mock(PulsarConfig.class); + when(config.isPopulateMetadata()).thenReturn(true); + when(config.getMetadataFields()).thenReturn(Set.of(MESSAGE_ID, MESSAGE_ID_BYTES_B64, MESSAGE_KEY)); DummyPulsarMessage pulsarMessage = new DummyPulsarMessage("key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8)); pulsarMessage.getProperties().put("test_key", "test_value"); pulsarMessage.getProperties().put("test_key2", "2"); - PulsarStreamMessageMetadata metadata = (PulsarStreamMessageMetadata) _metadataExtractor.extract(pulsarMessage); - assertEquals("test_value", metadata.getHeaders().getValue("test_key")); - assertEquals("2", metadata.getHeaders().getValue("test_key2")); - assertEquals("key", metadata.getRecordMetadata().get(MESSAGE_KEY.getKey())); - String messageIdStr = metadata.getRecordMetadata().get(MESSAGE_ID.getKey()); - assertEquals(pulsarMessage.getMessageId().toString(), messageIdStr); + StreamMessageMetadata metadata = PulsarUtils.extractMessageMetadata(pulsarMessage, config); + GenericRow headers = metadata.getHeaders(); + assertNotNull(headers); + assertEquals(headers.getValue("test_key"), "test_value"); + assertEquals(headers.getValue("test_key2"), "2"); + Map recordMetadata = metadata.getRecordMetadata(); + assertNotNull(recordMetadata); + assertEquals(recordMetadata.get(MESSAGE_KEY.getKey()), "key"); + assertEquals(recordMetadata.get(MESSAGE_ID.getKey()), pulsarMessage.getMessageId().toString()); - byte[] messageIdBytes = Base64.decode(metadata.getRecordMetadata().get(MESSAGE_ID_BYTES_B64.getKey())); + byte[] messageIdBytes = Base64.decode(recordMetadata.get(MESSAGE_ID_BYTES_B64.getKey())); MessageId messageId = MessageId.fromByteArray(messageIdBytes); assertEquals(MessageId.earliest, messageId); } @Test public void testPulsarSteamMessageUnstitched() { + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(false); String key = "key"; String value = "value"; DummyPulsarMessage dummyPulsarMessage = new DummyPulsarMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); - PulsarStreamMessage streamMessage = - PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, false, _metadataExtractor); - assertEquals(key.getBytes(StandardCharsets.UTF_8), streamMessage.getKey()); - assertEquals(value.getBytes(StandardCharsets.UTF_8), streamMessage.getValue()); - assertEquals(key.getBytes(StandardCharsets.UTF_8).length, streamMessage.getKeyLength()); - assertEquals(value.getBytes(StandardCharsets.UTF_8).length, streamMessage.getValueLength()); + BytesStreamMessage streamMessage = PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, config); + assertEquals(streamMessage.getKey(), key.getBytes(StandardCharsets.UTF_8)); + assertEquals(streamMessage.getValue(), value.getBytes(StandardCharsets.UTF_8)); } @Test public void testPulsarSteamMessageStitched() { + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(true); String key = "key"; String value = "value"; byte[] stitchedValueBytes = PulsarUtils.stitchKeyValue(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); DummyPulsarMessage dummyPulsarMessage = new DummyPulsarMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); - PulsarStreamMessage streamMessage = - PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, true, _metadataExtractor); - assertEquals(key.getBytes(StandardCharsets.UTF_8), streamMessage.getKey()); - assertEquals(stitchedValueBytes, streamMessage.getValue()); - assertEquals(key.getBytes(StandardCharsets.UTF_8).length, streamMessage.getKeyLength()); - assertEquals(stitchedValueBytes.length, streamMessage.getValueLength()); + BytesStreamMessage streamMessage = PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, config); + assertEquals(streamMessage.getKey(), key.getBytes(StandardCharsets.UTF_8)); + assertEquals(streamMessage.getValue(), stitchedValueBytes); } } diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java similarity index 71% rename from pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java index bab3ac045d91..1da5e2d7f1a0 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java @@ -16,10 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.jdbc; +package org.apache.pinot.calcite.jdbc; import java.util.List; import java.util.Map; +import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.schema.Function; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; @@ -28,26 +29,14 @@ /** * This class is used to create a {@link CalciteSchema} with a given {@link Schema} as the root. - * - *

This class resides in calcite.jdbc namespace because there's no complex logic we have in terms of catalog-based - * schema construct. We instead create a {@link SimpleCalciteSchema} that's package protected. */ public class CalciteSchemaBuilder { - private CalciteSchemaBuilder() { - // do not instantiate. } /** * Creates a {@link CalciteSchema} with a given {@link Schema} as the root. * - *

Calcite creates two layer of abstraction, the {@link CalciteSchema} is use internally for planner and - * {@link Schema} is user-facing with overrides. In our case we don't have a complex internal wrapper extension - * so we only reuse the package protected {@link SimpleCalciteSchema}. - * - *

If there's need to extend this feature for planner functionalities we should create our own extension to the - * {@link CalciteSchema}. - * * @param root schema to use as a root schema * @return calcite schema with given schema as the root */ diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java index 64d1e54e4db0..c345ca216cb1 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.prepare; +package org.apache.pinot.calcite.prepare; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -35,6 +35,9 @@ import org.apache.calcite.linq4j.function.Hints; import org.apache.calcite.model.ModelHandler; import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.prepare.Prepare; +import org.apache.calcite.prepare.RelOptTableImpl; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeFactoryImpl; @@ -78,16 +81,15 @@ /** - * ============================================================================ - * THIS CLASS IS COPIED FROM Calcite's {@link org.apache.calcite.prepare.CalciteCatalogReader} and modified the - * case sensitivity of Function lookup. which is ALWAYS case-insensitive regardless of conventions on - * column/table identifier. - * ============================================================================ + * ============================================================================================================== + * THIS CLASS IS COPIED FROM Calcite's {@link CalciteCatalogReader} and modified the case sensitivity of Function + * lookup, which is ALWAYS case-insensitive regardless of conventions on column/table identifier. + * ============================================================================================================== * - * Pinot's implementation of {@link org.apache.calcite.prepare.Prepare.CatalogReader} - * and also {@link org.apache.calcite.sql.SqlOperatorTable} based on tables and + * Pinot's implementation of {@link Prepare.CatalogReader} and also {@link SqlOperatorTable} based on tables and * functions defined schemas. */ +//@formatter:off public class PinotCalciteCatalogReader implements Prepare.CatalogReader { protected final CalciteSchema _rootSchema; protected final RelDataTypeFactory _typeFactory; @@ -463,3 +465,4 @@ private static RelDataType toSql(RelDataTypeFactory typeFactory, return null; } } +//@formatter:on diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java similarity index 95% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 0a7ec1b10a70..a45e92aba66f 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -16,15 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.hint; +package org.apache.pinot.calcite.rel.hint; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.hint.RelHint; import org.apache.pinot.query.planner.logical.LiteralHintUtils; /** * {@code PinotHintOptions} specified the supported hint options by Pinot based a particular type of relation node. * - *

for each {@link org.apache.calcite.rel.RelNode} type we support an option hint name.

+ *

for each {@link RelNode} type we support an option hint name.

*

for each option hint name there's a corresponding {@link RelHint} that supported only key-value option stored * in {@link RelHint#kvOptions}

*/ diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java similarity index 90% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java index c49a4b975403..524bffc75daf 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java @@ -16,13 +16,16 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.hint; +package org.apache.pinot.calcite.rel.hint; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.calcite.rel.hint.HintPredicates; +import org.apache.calcite.rel.hint.HintStrategyTable; +import org.apache.calcite.rel.hint.RelHint; import org.apache.pinot.spi.utils.BooleanUtils; @@ -30,17 +33,14 @@ * Default hint strategy set for Pinot query. */ public class PinotHintStrategyTable { - private PinotHintStrategyTable() { - // do not instantiate. } - public static final HintStrategyTable PINOT_HINT_STRATEGY_TABLE = HintStrategyTable.builder() - .hintStrategy(PinotHintOptions.INTERNAL_AGG_OPTIONS, HintPredicates.AGGREGATE) - .hintStrategy(PinotHintOptions.AGGREGATE_HINT_OPTIONS, HintPredicates.AGGREGATE) - .hintStrategy(PinotHintOptions.JOIN_HINT_OPTIONS, HintPredicates.JOIN) - .hintStrategy(PinotHintOptions.TABLE_HINT_OPTIONS, HintPredicates.TABLE_SCAN) - .build(); + public static final HintStrategyTable PINOT_HINT_STRATEGY_TABLE = + HintStrategyTable.builder().hintStrategy(PinotHintOptions.INTERNAL_AGG_OPTIONS, HintPredicates.AGGREGATE) + .hintStrategy(PinotHintOptions.AGGREGATE_HINT_OPTIONS, HintPredicates.AGGREGATE) + .hintStrategy(PinotHintOptions.JOIN_HINT_OPTIONS, HintPredicates.JOIN) + .hintStrategy(PinotHintOptions.TABLE_HINT_OPTIONS, HintPredicates.TABLE_SCAN).build(); /** * Check if a hint-able {@link org.apache.calcite.rel.RelNode} contains a specific {@link RelHint} by name. diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java similarity index 81% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java index c08b7ae8ade0..375a00500b78 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; @@ -36,16 +36,14 @@ public class PinotLogicalExchange extends Exchange { private final PinotRelExchangeType _exchangeType; - private PinotLogicalExchange(RelOptCluster cluster, RelTraitSet traitSet, - RelNode input, RelDistribution distribution, PinotRelExchangeType exchangeType) { + private PinotLogicalExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType) { super(cluster, traitSet, input, distribution); _exchangeType = exchangeType; assert traitSet.containsIfApplicable(Convention.NONE); } - - public static PinotLogicalExchange create(RelNode input, - RelDistribution distribution) { + public static PinotLogicalExchange create(RelNode input, RelDistribution distribution) { return create(input, distribution, PinotRelExchangeType.getDefaultExchangeType()); } @@ -56,22 +54,19 @@ public static PinotLogicalExchange create(RelNode input, * @param distribution Distribution specification * @param exchangeType RelExchangeType specification */ - public static PinotLogicalExchange create(RelNode input, - RelDistribution distribution, PinotRelExchangeType exchangeType) { + public static PinotLogicalExchange create(RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType) { RelOptCluster cluster = input.getCluster(); distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution); - RelTraitSet traitSet = - input.getTraitSet().replace(Convention.NONE).replace(distribution); + RelTraitSet traitSet = input.getTraitSet().replace(Convention.NONE).replace(distribution); return new PinotLogicalExchange(cluster, traitSet, input, distribution, exchangeType); } //~ Methods ---------------------------------------------------------------- @Override - public Exchange copy(RelTraitSet traitSet, RelNode newInput, - RelDistribution newDistribution) { - return new PinotLogicalExchange(getCluster(), traitSet, newInput, - newDistribution, _exchangeType); + public Exchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution) { + return new PinotLogicalExchange(getCluster(), traitSet, newInput, newDistribution, _exchangeType); } @Override diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java similarity index 78% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java index ace06a6c43f1..141b20d422f7 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; @@ -46,9 +46,9 @@ public class PinotLogicalSortExchange extends SortExchange { protected final boolean _isSortOnReceiver; protected final PinotRelExchangeType _exchangeType; - private PinotLogicalSortExchange(RelOptCluster cluster, RelTraitSet traitSet, - RelNode input, RelDistribution distribution, PinotRelExchangeType exchangeType, RelCollation collation, - boolean isSortOnSender, boolean isSortOnReceiver) { + private PinotLogicalSortExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDistribution distribution, PinotRelExchangeType exchangeType, RelCollation collation, boolean isSortOnSender, + boolean isSortOnReceiver) { super(cluster, traitSet, input, distribution, collation); _exchangeType = exchangeType; _isSortOnSender = isSortOnSender; @@ -65,12 +65,8 @@ public PinotLogicalSortExchange(RelInput input) { _isSortOnReceiver = true; } - public static PinotLogicalSortExchange create( - RelNode input, - RelDistribution distribution, - RelCollation collation, - boolean isSortOnSender, - boolean isSortOnReceiver) { + public static PinotLogicalSortExchange create(RelNode input, RelDistribution distribution, RelCollation collation, + boolean isSortOnSender, boolean isSortOnReceiver) { return create(input, distribution, PinotRelExchangeType.getDefaultExchangeType(), collation, isSortOnSender, isSortOnReceiver); } @@ -85,36 +81,29 @@ public static PinotLogicalSortExchange create( * @param isSortOnSender whether to sort on the sender * @param isSortOnReceiver whether to sort on receiver */ - public static PinotLogicalSortExchange create( - RelNode input, - RelDistribution distribution, - PinotRelExchangeType exchangeType, - RelCollation collation, - boolean isSortOnSender, - boolean isSortOnReceiver) { + public static PinotLogicalSortExchange create(RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType, RelCollation collation, boolean isSortOnSender, boolean isSortOnReceiver) { RelOptCluster cluster = input.getCluster(); collation = RelCollationTraitDef.INSTANCE.canonize(collation); distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution); - RelTraitSet traitSet = - input.getTraitSet().replace(Convention.NONE).replace(distribution).replace(collation); - return new PinotLogicalSortExchange(cluster, traitSet, input, distribution, exchangeType, - collation, isSortOnSender, isSortOnReceiver); + RelTraitSet traitSet = input.getTraitSet().replace(Convention.NONE).replace(distribution).replace(collation); + return new PinotLogicalSortExchange(cluster, traitSet, input, distribution, exchangeType, collation, isSortOnSender, + isSortOnReceiver); } //~ Methods ---------------------------------------------------------------- @Override - public SortExchange copy(RelTraitSet traitSet, RelNode newInput, - RelDistribution newDistribution, RelCollation newCollation) { - return new PinotLogicalSortExchange(this.getCluster(), traitSet, newInput, - newDistribution, _exchangeType, newCollation, _isSortOnSender, _isSortOnReceiver); + public SortExchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution, + RelCollation newCollation) { + return new PinotLogicalSortExchange(this.getCluster(), traitSet, newInput, newDistribution, _exchangeType, + newCollation, _isSortOnSender, _isSortOnReceiver); } @Override public RelWriter explainTerms(RelWriter pw) { - RelWriter relWriter = super.explainTerms(pw) - .item("isSortOnSender", _isSortOnSender) - .item("isSortOnReceiver", _isSortOnReceiver); + RelWriter relWriter = + super.explainTerms(pw).item("isSortOnSender", _isSortOnSender).item("isSortOnReceiver", _isSortOnReceiver); if (_exchangeType != PinotRelExchangeType.getDefaultExchangeType()) { relWriter.item("relExchangeType", _exchangeType); } diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java index 4a06e1d02562..49d5cff72839 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; /** Type of exchange. */ public enum PinotRelExchangeType { diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java index eca7fc61104a..7a7ae16c4da3 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java @@ -16,13 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; - -// NOTE: this file was generated using Calcite's code generator, but instead of pulling in all -// the dependencies for codegen we just manually generate it and check it in. If active development -// on this needs to happen, re-generate it using Calcite's generator. - -// CHECKSTYLE:OFF +package org.apache.pinot.calcite.rel.rules; import com.google.common.base.MoreObjects; import java.util.ArrayList; @@ -35,10 +29,16 @@ /** + * NOTE: this file was generated using Calcite's code generator, but instead of pulling in all + * the dependencies for codegen we just manually generate it and check it in. If active development + * on this needs to happen, re-generate it using Calcite's generator. + * * {@code ImmutableSortExchangeCopyRule} contains immutable implementation classes generated from * abstract value types defined as nested inside {@link SortExchangeCopyRule}. * @see ImmutableSortExchangeCopyRule.Config */ +// CHECKSTYLE:OFF +//@formatter:off @SuppressWarnings({"all"}) final class ImmutableSortExchangeCopyRule { private ImmutableSortExchangeCopyRule() { @@ -412,3 +412,4 @@ public ImmutableSortExchangeCopyRule.Config build() { } } } +//@formatter:on diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java index df904123d201..b7f0038145cb 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -33,14 +33,12 @@ import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.calcite.rel.logical.PinotLogicalExchange; +import org.apache.calcite.rel.rules.AggregateExtractProjectRule; +import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.PinotSqlAggFunction; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.OperandTypes; @@ -53,6 +51,10 @@ import org.apache.calcite.util.mapping.Mapping; import org.apache.calcite.util.mapping.MappingType; import org.apache.calcite.util.mapping.Mappings; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; +import org.apache.pinot.calcite.sql.PinotSqlAggFunction; import org.apache.pinot.query.planner.plannode.AggregateNode.AggType; import org.apache.pinot.segment.spi.AggregationFunctionType; @@ -62,7 +64,7 @@ * (1) non-data-locale Pinot server agg stage, and (2) the data-locale Pinot intermediate agg stage. * * Pinot uses special intermediate data representation for partially aggregated results, thus we can't use - * {@link org.apache.calcite.rel.rules.AggregateReduceFunctionsRule} to reduce complex aggregation. + * {@link AggregateReduceFunctionsRule} to reduce complex aggregation. * * This rule is here to introduces Pinot-special aggregation splits. In-general there are several options: *