Skip to content

Commit 4a70eb8

Browse files
authored
Always emit COMMITTING_SEGMENT_SIZE and NUM_ROWS_THRESHOLD regardless of the threshold update type (#18137)
1 parent f147b93 commit 4a70eb8

6 files changed

Lines changed: 232 additions & 288 deletions

File tree

pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java

Lines changed: 37 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -610,8 +610,7 @@ public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescrip
610610
// Copy the segment file to the controller
611611
String segmentLocation = committingSegmentDescriptor.getSegmentLocation();
612612
Preconditions.checkArgument(segmentLocation != null, "Segment location must be provided");
613-
if (segmentLocation.regionMatches(true, 0, CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME, 0,
614-
CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME.length())) {
613+
if (isPeerUrl(segmentLocation)) {
615614
LOGGER.info("No moving needed for segment on peer servers: {}", segmentLocation);
616615
return;
617616
}
@@ -635,6 +634,11 @@ public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescrip
635634
committingSegmentDescriptor.setSegmentLocation(uriToMoveTo);
636635
}
637636

637+
private boolean isPeerUrl(String segmentLocation) {
638+
return segmentLocation.regionMatches(true, 0, CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME, 0,
639+
CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME.length());
640+
}
641+
638642
/**
639643
* This method is invoked after the realtime segment is uploaded but before a response is sent to the server.
640644
* It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore
@@ -674,9 +678,9 @@ private void commitSegmentMetadataInternal(String realtimeTableName,
674678
// Step-1: Update PROPERTYSTORE
675679
LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName);
676680
long startTimeNs1 = System.nanoTime();
677-
SegmentZKMetadata committingSegmentZKMetadata =
678-
toCommitting ? updateSegmentZKMetadataToCommitting(realtimeTableName, committingSegmentDescriptor)
679-
: updateSegmentZKMetadataToDone(realtimeTableName, committingSegmentDescriptor, Status.IN_PROGRESS);
681+
SegmentZKMetadata committingSegmentZKMetadata = toCommitting
682+
? updateSegmentZKMetadataToCommitting(realtimeTableName, committingSegmentDescriptor)
683+
: updateSegmentZKMetadataToDone(realtimeTableName, committingSegmentDescriptor, Status.IN_PROGRESS);
680684

681685
preProcessNewSegmentZKMetadata();
682686

@@ -692,7 +696,7 @@ private void commitSegmentMetadataInternal(String realtimeTableName,
692696
LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName,
693697
newConsumingSegmentName);
694698
long startTimeNs3 = System.nanoTime();
695-
Map<String, Map<String, String>> instanceStatesMapAfterStep3 = Collections.emptyMap();
699+
Map<String, Map<String, String>> instanceStatesMapAfterStep3;
696700
boolean newConsumingSegmentInIdealState = false;
697701

698702
// When multiple segments of the same table complete around the same time it is possible that
@@ -791,8 +795,10 @@ private SegmentZKMetadata updateSegmentZKMetadataToDone(String realtimeTableName
791795
Preconditions.checkState(segmentMetadata != null, "Failed to find segment metadata from descriptor for segment: %s",
792796
segmentName);
793797
String segmentLocation = committingSegmentDescriptor.getSegmentLocation();
794-
String downloadUrl =
795-
isPeerURL(segmentLocation) ? CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD : segmentLocation;
798+
Preconditions.checkArgument(segmentLocation != null, "Segment location must be provided");
799+
String downloadUrl = isPeerUrl(segmentLocation)
800+
? CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD
801+
: segmentLocation;
796802
SegmentZKMetadataUtils.updateCommittingSegmentZKMetadata(realtimeTableName, segmentZKMetadata, segmentMetadata,
797803
downloadUrl, committingSegmentDescriptor.getSegmentSizeBytes(), committingSegmentDescriptor.getNextOffset());
798804
persistSegmentZKMetadata(realtimeTableName, segmentZKMetadata, stat.getVersion());
@@ -979,6 +985,7 @@ public void commitSegmentMetadataToDone(String realtimeTableName,
979985
FlushThresholdUpdater flushThresholdUpdater =
980986
_flushThresholdUpdateManager.getFlushThresholdUpdater(streamConfig);
981987
flushThresholdUpdater.onSegmentCommit(streamConfig, committingSegmentDescriptor, committingSegmentZKMetadata);
988+
updateCommittingSegmentSizeGauge(streamConfig, committingSegmentDescriptor.getSegmentSizeBytes());
982989
} catch (Exception e) {
983990
LOGGER.error("Caught exception while updating flush threshold for table: {}, segment: {}", realtimeTableName,
984991
segmentName, e);
@@ -988,11 +995,6 @@ public void commitSegmentMetadataToDone(String realtimeTableName,
988995
}
989996
}
990997

991-
private boolean isPeerURL(String segmentLocation) {
992-
return segmentLocation != null && segmentLocation.toLowerCase()
993-
.startsWith(CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME);
994-
}
995-
996998
/**
997999
* Creates and persists segment ZK metadata for the new CONSUMING segment.
9981000
*/
@@ -1041,13 +1043,35 @@ private void createNewSegmentZKMetadata(TableConfig tableConfig, StreamConfig st
10411043
FlushThresholdUpdater flushThresholdUpdater = _flushThresholdUpdateManager.getFlushThresholdUpdater(streamConfig);
10421044
if (committingSegmentZKMetadata != null) {
10431045
flushThresholdUpdater.onSegmentCommit(streamConfig, committingSegmentDescriptor, committingSegmentZKMetadata);
1046+
long segmentSize = committingSegmentDescriptor.getSegmentSizeBytes();
1047+
// Segment size might not be available for pauseless ingestion
1048+
if (segmentSize > 0) {
1049+
updateCommittingSegmentSizeGauge(streamConfig, segmentSize);
1050+
}
10441051
}
10451052
flushThresholdUpdater.updateFlushThreshold(streamConfig, newSegmentZKMetadata,
10461053
getMaxNumPartitionsPerInstance(instancePartitions, numPartitions, numReplicas));
1054+
updateFlushThresholdGauge(streamConfig, newSegmentZKMetadata.getSizeThresholdToFlushSegment());
10471055

10481056
persistSegmentZKMetadata(realtimeTableName, newSegmentZKMetadata, -1);
10491057
}
10501058

1059+
private void updateCommittingSegmentSizeGauge(StreamConfig streamConfig, long segmentSize) {
1060+
String realtimeTableName = streamConfig.getTableNameWithType();
1061+
String topicName = streamConfig.getTopicName();
1062+
_controllerMetrics.setOrUpdateTableGauge(realtimeTableName, ControllerGauge.COMMITTING_SEGMENT_SIZE, segmentSize);
1063+
_controllerMetrics.setOrUpdateTableGauge(realtimeTableName, topicName,
1064+
ControllerGauge.COMMITTING_SEGMENT_SIZE_WITH_TOPIC, segmentSize);
1065+
}
1066+
1067+
private void updateFlushThresholdGauge(StreamConfig streamConfig, int flushThreshold) {
1068+
String realtimeTableName = streamConfig.getTableNameWithType();
1069+
String topicName = streamConfig.getTopicName();
1070+
_controllerMetrics.setOrUpdateTableGauge(realtimeTableName, ControllerGauge.NUM_ROWS_THRESHOLD, flushThreshold);
1071+
_controllerMetrics.setOrUpdateTableGauge(realtimeTableName, topicName,
1072+
ControllerGauge.NUM_ROWS_THRESHOLD_WITH_TOPIC, flushThreshold);
1073+
}
1074+
10511075
private String computeStartOffset(String nextOffset, StreamConfig streamConfig, int partitionId) {
10521076
if (!streamConfig.isEnableOffsetAutoReset() || streamConfig.isBackfillTopic()) {
10531077
return nextOffset;

pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/CommittingSegmentDescriptor.java

Lines changed: 10 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
*/
1919
package org.apache.pinot.controller.helix.core.realtime.segment;
2020

21+
import javax.annotation.Nullable;
2122
import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
2223
import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
2324

@@ -26,10 +27,11 @@
2627
* Class to hold properties of the committing segment
2728
*/
2829
public class CommittingSegmentDescriptor {
29-
private String _segmentName;
30-
private long _segmentSizeBytes;
30+
private final String _segmentName;
31+
private final String _nextOffset;
32+
private final long _segmentSizeBytes;
33+
3134
private String _segmentLocation;
32-
private String _nextOffset;
3335
private SegmentMetadataImpl _segmentMetadata;
3436
private String _stopReason;
3537
private int _preCommitRowCount;
@@ -70,18 +72,15 @@ public String getSegmentName() {
7072
return _segmentName;
7173
}
7274

73-
public void setSegmentName(String segmentName) {
74-
_segmentName = segmentName;
75+
public String getNextOffset() {
76+
return _nextOffset;
7577
}
7678

7779
public long getSegmentSizeBytes() {
7880
return _segmentSizeBytes;
7981
}
8082

81-
public void setSegmentSizeBytes(long segmentSizeBytes) {
82-
_segmentSizeBytes = segmentSizeBytes;
83-
}
84-
83+
@Nullable
8584
public String getSegmentLocation() {
8685
return _segmentLocation;
8786
}
@@ -90,10 +89,7 @@ public void setSegmentLocation(String segmentLocation) {
9089
_segmentLocation = segmentLocation;
9190
}
9291

93-
public String getNextOffset() {
94-
return _nextOffset;
95-
}
96-
92+
@Nullable
9793
public SegmentMetadataImpl getSegmentMetadata() {
9894
return _segmentMetadata;
9995
}
@@ -102,6 +98,7 @@ public void setSegmentMetadata(SegmentMetadataImpl segmentMetadata) {
10298
_segmentMetadata = segmentMetadata;
10399
}
104100

101+
@Nullable
105102
public String getStopReason() {
106103
return _stopReason;
107104
}

pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdateManager.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,6 @@ public class FlushThresholdUpdateManager {
4747
*/
4848
public FlushThresholdUpdater getFlushThresholdUpdater(StreamConfig streamConfig) {
4949
String tableTopicKey = getKey(streamConfig);
50-
String realtimeTableName = streamConfig.getTableNameWithType();
5150
int flushThresholdRows = streamConfig.getFlushThresholdRows();
5251
if (flushThresholdRows > 0) {
5352
_flushThresholdUpdaterMap.remove(tableTopicKey);
@@ -61,8 +60,7 @@ public FlushThresholdUpdater getFlushThresholdUpdater(StreamConfig streamConfig)
6160
// Legacy behavior: when flush threshold rows is explicitly set to 0, use segment size based flush threshold
6261
long flushThresholdSegmentSizeBytes = streamConfig.getFlushThresholdSegmentSizeBytes();
6362
if (flushThresholdRows == 0 || flushThresholdSegmentSizeBytes > 0) {
64-
return _flushThresholdUpdaterMap.computeIfAbsent(tableTopicKey,
65-
k -> new SegmentSizeBasedFlushThresholdUpdater(realtimeTableName, streamConfig.getTopicName()));
63+
return _flushThresholdUpdaterMap.computeIfAbsent(tableTopicKey, k -> new SegmentSizeBasedFlushThresholdUpdater());
6664
} else {
6765
_flushThresholdUpdaterMap.remove(tableTopicKey);
6866
return new DefaultFlushThresholdUpdater(StreamConfig.DEFAULT_FLUSH_THRESHOLD_ROWS);

pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/SegmentSizeBasedFlushThresholdUpdater.java

Lines changed: 1 addition & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,6 @@
1919
package org.apache.pinot.controller.helix.core.realtime.segment;
2020

2121
import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
22-
import org.apache.pinot.common.metrics.ControllerGauge;
23-
import org.apache.pinot.common.metrics.ControllerMetrics;
2422
import org.apache.pinot.spi.stream.StreamConfig;
2523

2624

@@ -34,25 +32,14 @@
3432
*/
3533
public class SegmentSizeBasedFlushThresholdUpdater implements FlushThresholdUpdater {
3634
private final SizeBasedSegmentFlushThresholdComputer _flushThresholdComputer;
37-
private final String _realtimeTableName;
38-
private final String _topicName;
3935

40-
private final ControllerMetrics _controllerMetrics = ControllerMetrics.get();
41-
42-
public SegmentSizeBasedFlushThresholdUpdater(String realtimeTableName, String topicName) {
36+
public SegmentSizeBasedFlushThresholdUpdater() {
4337
_flushThresholdComputer = new SizeBasedSegmentFlushThresholdComputer();
44-
_realtimeTableName = realtimeTableName;
45-
_topicName = topicName;
4638
}
4739

4840
@Override
4941
public void onSegmentCommit(StreamConfig streamConfig, CommittingSegmentDescriptor committingSegmentDescriptor,
5042
SegmentZKMetadata committingSegmentZKMetadata) {
51-
long segmentSize = committingSegmentDescriptor.getSegmentSizeBytes();
52-
_controllerMetrics.setOrUpdateTableGauge(_realtimeTableName, ControllerGauge.COMMITTING_SEGMENT_SIZE, segmentSize);
53-
_controllerMetrics.setOrUpdateTableGauge(_realtimeTableName, _topicName,
54-
ControllerGauge.COMMITTING_SEGMENT_SIZE_WITH_TOPIC, segmentSize);
55-
5643
_flushThresholdComputer.onSegmentCommit(committingSegmentDescriptor, committingSegmentZKMetadata);
5744
}
5845

@@ -61,9 +48,5 @@ public void updateFlushThreshold(StreamConfig streamConfig, SegmentZKMetadata ne
6148
int maxNumPartitionsPerInstance) {
6249
int threshold = _flushThresholdComputer.computeThreshold(streamConfig, newSegmentZKMetadata.getSegmentName());
6350
newSegmentZKMetadata.setSizeThresholdToFlushSegment(threshold);
64-
65-
_controllerMetrics.setOrUpdateTableGauge(_realtimeTableName, ControllerGauge.NUM_ROWS_THRESHOLD, threshold);
66-
_controllerMetrics.setOrUpdateTableGauge(_realtimeTableName, _topicName,
67-
ControllerGauge.NUM_ROWS_THRESHOLD_WITH_TOPIC, threshold);
6851
}
6952
}

0 commit comments

Comments
 (0)