Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support reloading consuming segment using force commit #9640

Merged
merged 1 commit into from
Oct 27, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,7 @@ public void addSegment(ImmutableSegment immutableSegment) {
}

// TODO: Change dedup handling to handle segment replacement
if (isDedupEnabled()) {
if (isDedupEnabled() && immutableSegment instanceof ImmutableSegmentImpl) {
buildDedupMeta((ImmutableSegmentImpl) immutableSegment);
}
super.addSegment(immutableSegment);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ protected long getCountStarResult() {
}

protected boolean useLlc() {
return false;
return true;
}

protected boolean useKafkaTransaction() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,12 +83,6 @@ public void setUp()
waitForAllDocsLoaded(600_000L);
}

@Override
protected boolean useLlc() {
// NOTE: Aggregate metrics is only available with LLC.
return true;
}

@Override
protected void waitForAllDocsLoaded(long timeoutMs) {
// NOTE: For aggregate metrics, we need to test the aggregation result instead of the document count because
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
/**
* Integration test that creates a Kafka broker, creates a Pinot cluster that consumes from Kafka and queries Pinot.
*/
public class RealtimeClusterIntegrationTest extends BaseClusterIntegrationTestSet {
public abstract class BaseRealtimeClusterIntegrationTest extends BaseClusterIntegrationTestSet {

@BeforeClass
public void setUp()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,11 +79,6 @@ protected String getTableName() {
return _currentTable;
}

@Override
protected boolean useLlc() {
return true;
}

@Override
protected int getNumReplicas() {
return NUM_REPLICAS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,11 +96,6 @@ protected String getAvroTarFileName() {
return "dedupIngestionTestData.tar.gz";
}

@Override
protected boolean useLlc() {
return true;
}

@Override
protected String getPartitionColumn() {
return "id";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,7 @@
import org.apache.pinot.spi.utils.ReadMode;


public class ExactlyOnceKafkaRealtimeClusterIntegrationTest extends RealtimeClusterIntegrationTest {

@Override
protected boolean useLlc() {
return true;
}
public class ExactlyOnceKafkaRealtimeClusterIntegrationTest extends BaseRealtimeClusterIntegrationTest {

@Override
protected boolean useKafkaTransaction() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
/**
* Integration test that simulates a flaky Kafka consumer.
*/
public class FlakyConsumerRealtimeClusterIntegrationTest extends RealtimeClusterIntegrationTest {
public class FlakyConsumerHLCRealtimeClusterIntegrationTest extends HLCRealtimeClusterIntegrationTest {

@Override
protected String getStreamConsumerFactoryClassName() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/**
* 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.integration.tests;

/**
* Integration test for high-level Kafka consumer.
*/
public class HLCRealtimeClusterIntegrationTest extends BaseRealtimeClusterIntegrationTest {

@Override
protected boolean useLlc() {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -135,51 +135,39 @@ public void testSegmentMetadataApi()
segmentMetadataFromDirectEndpoint.get("segment.total.docs"));
}

// TODO: This test fails when using `llc` consumer mode. Needs investigation
@Test
public void testSegmentListApi()
throws Exception {
{
String jsonOutputStr = sendGetRequest(
_controllerRequestURLBuilder.forSegmentListAPI(getTableName(), TableType.OFFLINE.toString()));
String jsonOutputStr =
sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPI(getTableName(), TableType.OFFLINE.toString()));
JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
// There should be one element in the array
JsonNode element = array.get(0);
JsonNode segments = element.get("OFFLINE");
Assert.assertEquals(segments.size(), 8);
}
{
String jsonOutputStr = sendGetRequest(
_controllerRequestURLBuilder.forSegmentListAPI(getTableName(), TableType.REALTIME.toString()));
String jsonOutputStr =
sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPI(getTableName(), TableType.REALTIME.toString()));
JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
// There should be one element in the array
JsonNode element = array.get(0);
JsonNode segments = element.get("REALTIME");
Assert.assertEquals(segments.size(), 3);
Assert.assertEquals(segments.size(), 24);
}
{
String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPI(getTableName()));
JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
// there should be 2 elements in the array now.
int realtimeIndex = 0;
int offlineIndex = 1;
JsonNode element = array.get(realtimeIndex);
if (!element.has("REALTIME")) {
realtimeIndex = 1;
offlineIndex = 0;
}
JsonNode offlineElement = array.get(offlineIndex);
JsonNode realtimeElement = array.get(realtimeIndex);

JsonNode realtimeSegments = realtimeElement.get("REALTIME");
Assert.assertEquals(realtimeSegments.size(), 3);

JsonNode offlineSegments = offlineElement.get("OFFLINE");
JsonNode offlineSegments = array.get(0).get("OFFLINE");
Assert.assertEquals(offlineSegments.size(), 8);
JsonNode realtimeSegments = array.get(1).get("REALTIME");
Assert.assertEquals(realtimeSegments.size(), 24);
}
}

@Test
// NOTE: Reload consuming segment will force commit it, so run this test after segment list api test
@Test(dependsOnMethods = "testSegmentListApi")
public void testReload()
throws Exception {
super.testReload(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@
* TODO: Add separate module-level tests and remove the randomness of this test
*/
public class KafkaConfluentSchemaRegistryAvroMessageDecoderRealtimeClusterIntegrationTest
extends RealtimeClusterIntegrationTest {
extends BaseRealtimeClusterIntegrationTest {
private static final String CONSUMER_DIRECTORY = "/tmp/consumer-test";
private static final String TEST_UPDATED_INVERTED_INDEX_QUERY =
"SELECT COUNT(*) FROM mytable WHERE DivActualElapsedTime = 305";
Expand Down Expand Up @@ -185,11 +185,6 @@ protected boolean injectTombstones() {
return true;
}

@Override
protected boolean useLlc() {
return true;
}

@Override
protected String getLoadMode() {
return ReadMode.mmap.name();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,10 +55,10 @@


/**
* Integration test that extends RealtimeClusterIntegrationTest but uses low-level Kafka consumer.
* Integration test for low-level Kafka consumer.
* TODO: Add separate module-level tests and remove the randomness of this test
*/
public class LLCRealtimeClusterIntegrationTest extends RealtimeClusterIntegrationTest {
public class LLCRealtimeClusterIntegrationTest extends BaseRealtimeClusterIntegrationTest {
private static final String CONSUMER_DIRECTORY = "/tmp/consumer-test";
private static final String TEST_UPDATED_INVERTED_INDEX_QUERY =
"SELECT COUNT(*) FROM mytable WHERE DivActualElapsedTime = 305";
Expand All @@ -77,11 +77,6 @@ protected boolean injectTombstones() {
return true;
}

@Override
protected boolean useLlc() {
return true;
}

@Override
protected String getLoadMode() {
return ReadMode.mmap.name();
Expand Down Expand Up @@ -155,17 +150,17 @@ private void uploadSegmentsToController(String tableName, File tarDir, boolean o
if (changeCrc) {
changeCrcInSegmentZKMetadata(tableName, segmentTarFile.toString());
}
assertEquals(fileUploadDownloadClient
.uploadSegment(uploadSegmentHttpURI, segmentTarFile.getName(), segmentTarFile, tableName,
TableType.REALTIME).getStatusCode(), HttpStatus.SC_OK);
assertEquals(
fileUploadDownloadClient.uploadSegment(uploadSegmentHttpURI, segmentTarFile.getName(), segmentTarFile,
tableName, TableType.REALTIME).getStatusCode(), HttpStatus.SC_OK);
} else {
// Upload segments in parallel
ExecutorService executorService = Executors.newFixedThreadPool(numSegments);
List<Future<Integer>> futures = new ArrayList<>(numSegments);
for (File segmentTarFile : segmentTarFiles) {
futures.add(executorService.submit(() -> fileUploadDownloadClient
.uploadSegment(uploadSegmentHttpURI, segmentTarFile.getName(), segmentTarFile, tableName,
TableType.REALTIME).getStatusCode()));
futures.add(executorService.submit(
() -> fileUploadDownloadClient.uploadSegment(uploadSegmentHttpURI, segmentTarFile.getName(),
segmentTarFile, tableName, TableType.REALTIME).getStatusCode()));
}
executorService.shutdown();
for (Future<Integer> future : futures) {
Expand Down Expand Up @@ -253,10 +248,12 @@ public void testInvertedIndexTriggering()
JsonNode queryResponse1 = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
// Total docs should not change during reload
assertEquals(queryResponse1.get("totalDocs").asLong(), numTotalDocs);
assertEquals(queryResponse1.get("numConsumingSegmentsQueried").asLong(), 2);
assertTrue(queryResponse1.get("minConsumingFreshnessTimeMs").asLong() > _startTime);
assertTrue(queryResponse1.get("minConsumingFreshnessTimeMs").asLong() < System.currentTimeMillis());
return queryResponse1.get("numEntriesScannedInFilter").asLong() == 0;

long numConsumingSegmentsQueried = queryResponse1.get("numConsumingSegmentsQueried").asLong();
long minConsumingFreshnessTimeMs = queryResponse1.get("minConsumingFreshnessTimeMs").asLong();
long numEntriesScannedInFilter = queryResponse1.get("numEntriesScannedInFilter").asLong();
return numConsumingSegmentsQueried == 2 && minConsumingFreshnessTimeMs > _startTime
&& minConsumingFreshnessTimeMs < System.currentTimeMillis() && numEntriesScannedInFilter == 0;
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@
* (1) All the segments on all servers are in either ONLINE or CONSUMING states
* (2) For segments failed during deep store upload, the corresponding segment download url string is empty in Zk.
*/
public class PeerDownloadLLCRealtimeClusterIntegrationTest extends RealtimeClusterIntegrationTest {
public class PeerDownloadLLCRealtimeClusterIntegrationTest extends BaseRealtimeClusterIntegrationTest {
private static final Logger LOGGER = LoggerFactory.getLogger(PeerDownloadLLCRealtimeClusterIntegrationTest.class);

private static final String CONSUMER_DIRECTORY = "/tmp/consumer-test";
Expand Down Expand Up @@ -151,11 +151,6 @@ public void startController()
enableResourceConfigForLeadControllerResource(_enableLeadControllerResource);
}

@Override
protected boolean useLlc() {
return true;
}

@Nullable
@Override
protected String getLoadMode() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@
* With every task run, a new segment is created in the offline table for 1 day. Watermark also keeps progressing
* accordingly.
*/
public class RealtimeToOfflineSegmentsMinionClusterIntegrationTest extends RealtimeClusterIntegrationTest {
public class RealtimeToOfflineSegmentsMinionClusterIntegrationTest extends BaseRealtimeClusterIntegrationTest {

private PinotHelixTaskResourceManager _helixTaskResourceManager;
private PinotTaskManager _taskManager;
Expand All @@ -67,11 +67,6 @@ protected TableTaskConfig getTaskConfig() {
Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, new HashMap<>()));
}

@Override
protected boolean useLlc() {
return true;
}

@BeforeClass
public void setUp()
throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,11 +61,6 @@ public class SegmentCompletionIntegrationTest extends BaseClusterIntegrationTest
private HelixManager _serverHelixManager;
private String _currentSegment;

@Override
protected boolean useLlc() {
return true;
}

@Override
protected int getNumKafkaPartitions() {
return NUM_KAFKA_PARTITIONS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,11 +110,6 @@ protected long getCountStarResult() {
return _countStarResult;
}

@Override
protected boolean useLlc() {
return true;
}

@Nullable
@Override
protected String getPartitionColumn() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,12 +72,6 @@ public String getTimeColumnName() {
return TIME_COLUMN_NAME;
}

// TODO: Support Lucene index on HLC consuming segments
@Override
protected boolean useLlc() {
return true;
}

@Nullable
@Override
protected String getSortedColumn() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,11 +242,6 @@ protected TableTaskConfig getTaskConfig() {
return new TableTaskConfig(Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, prop));
}

@Override
protected boolean useLlc() {
return true;
}

@Override
public void addSchema(Schema schema)
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,11 +129,6 @@ protected String getAvroTarFileName() {
return "upsert_test.tar.gz";
}

@Override
protected boolean useLlc() {
return true;
}

@Override
protected String getPartitionColumn() {
return PRIMARY_KEY_COL;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,11 +152,6 @@ protected TableTaskConfig getTaskConfig() {
Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, properties));
}

@Override
protected boolean useLlc() {
return true;
}

@Override
public void addSchema(Schema schema)
throws IOException {
Expand Down
Loading