forked from apache/camel-kafka-connector
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
apache#1662 Implementation of the EventHubOffsetCheckpointStore
- Loading branch information
1 parent
2003ae6
commit 4b5bfaa
Showing
11 changed files
with
688 additions
and
79 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
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
74 changes: 74 additions & 0 deletions
74
...apache/camel/kafkaconnector/azureeventhubssource/checkpoint/EventHubCheckpointOffset.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,74 @@ | ||
/* | ||
* 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.camel.kafkaconnector.azureeventhubssource.checkpoint; | ||
|
||
import static java.util.Collections.unmodifiableMap; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import com.azure.messaging.eventhubs.models.Checkpoint; | ||
|
||
/** | ||
* The source offset information about the Azure EventHub checkpoint. | ||
*/ | ||
record EventHubCheckpointOffset(Map<String, Object> value) { | ||
private static final Logger LOG = LoggerFactory.getLogger(EventHubOffsetCheckpointStore.class); | ||
|
||
private static final String OFFSET = "offset"; | ||
private static final String SEQUENCE_NUMBER = "sequenceNo"; | ||
|
||
static EventHubCheckpointOffset of(Checkpoint checkpoint) { | ||
var value = new HashMap<String, Object>(); | ||
Optional.ofNullable(checkpoint.getOffset()).ifPresent(it -> value.put(OFFSET, it)); | ||
Optional.ofNullable(checkpoint.getSequenceNumber()).ifPresent(it -> value.put(SEQUENCE_NUMBER, it)); | ||
return new EventHubCheckpointOffset(unmodifiableMap(value)); | ||
} | ||
|
||
static Optional<EventHubCheckpointOffset> tryParse(final Map<?, ?> value) { | ||
final var result = new HashMap<String, Object>(); | ||
if (value.get(OFFSET) instanceof Number offset) { | ||
result.put(OFFSET, offset.longValue()); | ||
} | ||
if (value.get(SEQUENCE_NUMBER) instanceof Number sequenceNumber) { | ||
result.put(SEQUENCE_NUMBER, sequenceNumber.longValue()); | ||
} | ||
if (result.isEmpty()) { | ||
LOG.warn("Invalid checkpoint-value {}", value); | ||
return Optional.empty(); | ||
} | ||
return Optional.of(new EventHubCheckpointOffset(unmodifiableMap(result))); | ||
} | ||
|
||
Long offset() { | ||
if (value.get(OFFSET) instanceof Long offset) { | ||
return offset; | ||
} | ||
return null; | ||
} | ||
|
||
Long sequenceNumber() { | ||
if (value.get(SEQUENCE_NUMBER) instanceof Long sequenceNumber) { | ||
return sequenceNumber; | ||
} | ||
return null; | ||
} | ||
} |
92 changes: 92 additions & 0 deletions
92
...org/apache/camel/kafkaconnector/azureeventhubssource/checkpoint/EventHubConsumerInfo.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,92 @@ | ||
/* | ||
* 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.camel.kafkaconnector.azureeventhubssource.checkpoint; | ||
|
||
import static java.util.Collections.unmodifiableMap; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import com.azure.messaging.eventhubs.models.Checkpoint; | ||
import com.azure.messaging.eventhubs.models.PartitionOwnership; | ||
|
||
/** | ||
* The source offset information about the Azure EventHub consumer. | ||
*/ | ||
record EventHubConsumerInfo(Map<String, String> value) { | ||
private static final Logger LOG = LoggerFactory.getLogger(EventHubOffsetCheckpointStore.class); | ||
|
||
private static final String NAMESPACE = "namespace"; | ||
private static final String EVENT_HUB = "eventHub"; | ||
private static final String CONSUMER_GROUP = "consumerGroup"; | ||
|
||
static EventHubConsumerInfo of(String namespace, String eventHub, String consumerGroup) { | ||
var value = new HashMap<String, String>(); | ||
Optional.ofNullable(namespace).ifPresent(it -> value.put(NAMESPACE, it)); | ||
Optional.ofNullable(eventHub).ifPresent(it -> value.put(EVENT_HUB, it)); | ||
Optional.ofNullable(consumerGroup).ifPresent(it -> value.put(CONSUMER_GROUP, it)); | ||
return new EventHubConsumerInfo(unmodifiableMap(value)); | ||
} | ||
|
||
static EventHubConsumerInfo of(Checkpoint checkpoint) { | ||
return of(checkpoint.getFullyQualifiedNamespace(), | ||
checkpoint.getEventHubName(), | ||
checkpoint.getConsumerGroup()); | ||
} | ||
|
||
static EventHubConsumerInfo of(PartitionOwnership partitionOwnership) { | ||
return of(partitionOwnership.getFullyQualifiedNamespace(), | ||
partitionOwnership.getEventHubName(), | ||
partitionOwnership.getConsumerGroup()); | ||
} | ||
|
||
static Optional<EventHubConsumerInfo> tryParse(Map<?, ?> value) { | ||
String fullyQualifiedNamespace = null; | ||
String eventHubName = null; | ||
String consumerGroup = null; | ||
if (value.get(NAMESPACE) instanceof String it) { | ||
fullyQualifiedNamespace = it; | ||
} | ||
if (value.get(NAMESPACE) instanceof String it) { | ||
eventHubName = it; | ||
} | ||
if (value.get(CONSUMER_GROUP) instanceof String it) { | ||
consumerGroup = it; | ||
} | ||
if (fullyQualifiedNamespace == null && eventHubName == null && consumerGroup == null) { | ||
LOG.warn("Invalid offset-key {}", value); | ||
return Optional.empty(); | ||
} | ||
return Optional.of(of(fullyQualifiedNamespace, eventHubName, consumerGroup)); | ||
} | ||
|
||
String namespace() { | ||
return value.get(NAMESPACE); | ||
} | ||
|
||
String eventHub() { | ||
return value.get(EVENT_HUB); | ||
} | ||
|
||
String consumerGroup() { | ||
return value.get(CONSUMER_GROUP); | ||
} | ||
} |
Oops, something went wrong.