-
Notifications
You must be signed in to change notification settings - Fork 139
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
12 changed files
with
359 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
62 changes: 62 additions & 0 deletions
62
parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/Offsets.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
package io.confluent.parallelconsumer; | ||
|
||
/*- | ||
* Copyright (C) 2020-2022 Confluent, Inc. | ||
*/ | ||
|
||
import lombok.Getter; | ||
import lombok.ToString; | ||
import lombok.Value; | ||
import lombok.experimental.Delegate; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.List; | ||
import java.util.SortedSet; | ||
import java.util.stream.Collectors; | ||
|
||
/** | ||
* A range of {@link Offset}s. | ||
*/ | ||
@ToString | ||
public class Offsets { | ||
|
||
@Getter | ||
@Delegate | ||
private final List<Long> rawOffsets; | ||
|
||
public Offsets(List<Long> records) { | ||
this.rawOffsets = records; | ||
} | ||
|
||
public static Offsets fromRecords(List<RecordContext<?, ?>> records) { | ||
return fromLongs(records.stream() | ||
.map(RecordContext::offset) | ||
.collect(Collectors.toUnmodifiableList())); | ||
} | ||
|
||
// due to type erasure, can't use method overloading | ||
public static Offsets fromLongs(List<Long> rawOffsetsIn) { | ||
return new Offsets(rawOffsetsIn); | ||
} | ||
|
||
public static Offsets fromArray(long... rawOffsetsIn) { | ||
return new Offsets(Arrays.stream(rawOffsetsIn).boxed().collect(Collectors.toList())); | ||
} | ||
|
||
public static Offsets from(SortedSet<Long> incompleteOffsetsBelowHighestSucceeded) { | ||
return new Offsets(new ArrayList<>(incompleteOffsetsBelowHighestSucceeded)); | ||
} | ||
|
||
/** | ||
* Class value for a Kafka offset, to avoid being Longly typed. | ||
*/ | ||
@Value | ||
public static class Offset { | ||
long value; | ||
|
||
public static Offset of(Long offset) { | ||
return new Offset(offset); | ||
} | ||
} | ||
} |
113 changes: 113 additions & 0 deletions
113
parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/PCMetrics.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,113 @@ | ||
package io.confluent.parallelconsumer; | ||
|
||
import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; | ||
import io.confluent.parallelconsumer.internal.State; | ||
import io.confluent.parallelconsumer.offsets.OffsetEncoding; | ||
import io.confluent.parallelconsumer.state.ShardKey; | ||
import lombok.Builder; | ||
import lombok.Value; | ||
import lombok.experimental.SuperBuilder; | ||
import org.apache.kafka.common.TopicPartition; | ||
|
||
import java.util.Map; | ||
import java.util.Optional; | ||
|
||
/** | ||
* Metrics model for Parallel Consumer | ||
* | ||
* @author Antony Stubbs | ||
*/ | ||
@Value | ||
@SuperBuilder(toBuilder = true) | ||
public class PCMetrics { | ||
|
||
long dynamicLoadFactor; | ||
|
||
Map<TopicPartition, PCPartitionMetrics> partitionMetrics; | ||
|
||
Map<ShardKey, ShardMetrics> shardMetrics; | ||
|
||
PollerMetrics pollerMetrics; | ||
|
||
/** | ||
* The number of partitions assigned to this consumer | ||
*/ | ||
public long getNumberOfPartitions() { | ||
return partitionMetrics.size(); | ||
} | ||
|
||
/** | ||
* The number of shards (queues) currently managed - depends on ordering and data key set | ||
*/ | ||
public long getNumberOfShards() { | ||
return shardMetrics.size(); | ||
} | ||
|
||
public long getTotalNumberOfIncompletes() { | ||
return partitionMetrics.values().stream() | ||
.mapToLong(PCPartitionMetrics::getNumberOfIncompletes) | ||
.sum(); | ||
} | ||
|
||
@Value | ||
@SuperBuilder(toBuilder = true) | ||
public static class PCPartitionMetrics { | ||
TopicPartition topicPartition; | ||
long lastCommittedOffset; | ||
long numberOfIncompletes; | ||
long highestCompletedOffset; | ||
long highestSeenOffset; | ||
long highestSequentialSucceededOffset; | ||
long epoch; | ||
CompressionStats compressionStats; | ||
|
||
/** | ||
* @see AbstractParallelEoSStreamProcessor#calculateMetricsWithIncompletes() | ||
*/ | ||
@Builder.Default | ||
Optional<IncompleteMetrics> incompleteMetrics = Optional.empty(); | ||
|
||
public long getTraditionalConsumerLag() { | ||
return highestSeenOffset - highestSequentialSucceededOffset; | ||
} | ||
|
||
@Value | ||
public static class IncompleteMetrics { | ||
Offsets incompleteOffsets; | ||
} | ||
|
||
@Value | ||
@SuperBuilder(toBuilder = true) | ||
public static class CompressionStats { | ||
long offsetsEncodedPerBit; | ||
long offsetsEncodedPerByte; | ||
long bytesUsedForEncoding; | ||
double fractionOfEncodingSpaceUsed; | ||
OffsetEncoding bestEncoding; | ||
} | ||
} | ||
|
||
@Value | ||
@SuperBuilder(toBuilder = true) | ||
public static class ShardMetrics { | ||
ShardKey shardKey; | ||
/** | ||
* Number of records queued for processing in this shard | ||
*/ | ||
long shardSize; | ||
long averageUserProcessingTime; | ||
long averageTimeSpentInQueue; | ||
} | ||
|
||
/** | ||
* Metrics for the {@link io.confluent.parallelconsumer.internal.BrokerPollSystem} sub-system. | ||
*/ | ||
@Value | ||
@SuperBuilder(toBuilder = true) | ||
public static class PollerMetrics { | ||
State state; | ||
boolean paused; | ||
Map<TopicPartition, Boolean> pausedPartitions; | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.