frankgh commented on code in PR #171:
URL: https://github.com/apache/cassandra-sidecar/pull/171#discussion_r1953389945
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRange.java:
##########
@@ -202,4 +265,35 @@ public int hashCode()
{
return range.hashCode();
}
+
+ @Override
+ public String toString()
+ {
+ return "TokenRange(" +
+ range.lowerEndpoint().toBigInteger() + ", " +
+ range.upperEndpoint().toBigInteger() + ']';
+ }
+
+ /**
+ * Pair of {@link TokenRange} sets
+ */
+ public static class Pair
+ {
+ public final ImmutableSet<TokenRange> left;
+ public final ImmutableSet<TokenRange> right;
+
+ private Pair(Set<Range<Token>> left, Set<Range<Token>> right)
+ {
+ this.left = toImmutableSet(left);
+ this.right = toImmutableSet(right);
Review Comment:
now we can use Set.of since we've moved to java 11. These are immutable
(unmodifiable) sets.
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRange.java:
##########
@@ -202,4 +265,35 @@ public int hashCode()
{
return range.hashCode();
}
+
+ @Override
+ public String toString()
+ {
+ return "TokenRange(" +
+ range.lowerEndpoint().toBigInteger() + ", " +
+ range.upperEndpoint().toBigInteger() + ']';
+ }
+
+ /**
+ * Pair of {@link TokenRange} sets
+ */
+ public static class Pair
Review Comment:
should this class be called diff instead, and maybe name the `left` and
`right` fields to something more appropriate? It only seems to be used by the
diff method
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRange.java:
##########
@@ -108,6 +140,14 @@ public Token start()
return range.lowerEndpoint();
}
+ /**
+ * @return start token as {@link BigInteger}. It is not enclosed in the
range.
+ */
+ public BigInteger startAsBigInt()
Review Comment:
also seeing usage of range.lowerEndpoint.toBigInteger from callsites, do we
want to use this method instead from those callsites?
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/exceptions/SchemaUnavailableException.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.cassandra.sidecar.common.server.exceptions;
+
+import com.datastax.driver.core.PreparedStatement;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception thrown when {@link
org.apache.cassandra.sidecar.db.schema.TableSchema} does not exist.
+ * For instance, the connected Cassandra no longer has such table
+ */
+public class SchemaUnavailableException extends RuntimeException
+{
+ /**
+ * Throws {@link SchemaUnavailableException} if {@link PreparedStatement}
is null
+ * @param statement prepared statement to check
+ * @param errorMessage error message to use when the prepared statement is
null
+ */
+ public static void requirePrepared(@Nullable PreparedStatement statement,
String errorMessage)
Review Comment:
unused?
##########
server-common/src/test/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRangeTest.java:
##########
@@ -159,6 +165,66 @@ void testIntersection()
assertThat(r2.intersection(r4)).isNotEqualTo(new TokenRange(5, 5)); //
but not any empty range
}
+ @ParameterizedTest(name = "{index} - {0}: inputLeft={1} inputRight={2}
expectedLeft={3} expectedRight={4}")
+ @MethodSource("inputAndExpectedResultAfterDiff")
+ void testDiff(String testTitle, Set<TokenRange> left, Set<TokenRange>
right, Set<TokenRange> expectedLeft, Set<TokenRange> expectedRight)
Review Comment:
did you mean to use `testTitle` somewhere?
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RestoreJobManagerGroup.java:
##########
@@ -114,6 +122,29 @@ void updateRestoreJob(RestoreJob restoreJob)
managerGroup.values().forEach(manager ->
manager.updateRestoreJob(restoreJob));
}
+ /**
+ * Discard the ranges that overlap with the given {@param otherRanges}
+ * @param instanceMetadata cassandra instance to discard the restore range
from
+ * @param restoreJob restore job instance
+ * @param otherRanges set of {@link TokenRange} to find the overlapping
{@link RestoreRange} and discard
+ * @return set of overlapping {@link RestoreRange}
+ */
+ Set<RestoreRange> discardOverlappingRanges(InstanceMetadata
instanceMetadata, RestoreJob restoreJob, Set<TokenRange> otherRanges)
+ {
+ if (restoreJob.status.isFinal())
+ {
+ throw new IllegalStateException("Cannot remove ranges from a
restore job in final status");
+ }
+ RestoreJobManager manager = managerGroup.get(instanceMetadata.id());
+ if (manager == null)
+ {
+ LOGGER.debug("No RestoreJobManager found for Cassandra instance.
No ranges to discard. instanceId={}",
+ instanceMetadata.id());
+ return Collections.emptySet();
Review Comment:
NIT: Set.of
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RingTopologyRefresher.java:
##########
@@ -104,6 +153,86 @@ public TokenRangeReplicasResponse
cachedReplicaByTokenRange(RestoreJob restoreJo
return replicaByTokenRangePerKeyspace.forRestoreJob(restoreJob);
}
+ /**
+ * Fetch the latest topology view
+ * <p>It is synchronized when force refreshing as there is potential
contention from {@link #execute(Promise)}
+ *
+ * @param keyspace keyspace to determine replication
+ * @param forceRefresh whether refresh the topology view forcibly or not
+ * @return token ranges of the local Cassandra instances or an empty map
of nothing is found
+ */
+ @Override
+ public Map<Integer, Set<TokenRange>> localTokenRanges(String keyspace,
boolean forceRefresh)
+ {
+ TokenRangeReplicasResponse topology;
+ if (forceRefresh) // fetch the latest topology and load into cache
+ {
+ synchronized (this)
+ {
+ topology = prepareAndFetch((storageOperations, nodeSettings)
-> {
+ String partitioner = nodeSettings.partitioner();
+ return replicaByTokenRangePerKeyspace.loadOne(keyspace, k
-> storageOperations.tokenRangeReplicas(new Name(keyspace), partitioner));
+ });
+ }
+ }
+ else // get the cached value
+ {
+ topology =
replicaByTokenRangePerKeyspace.topologyOfKeyspace(keyspace);
+ }
+
+ return calculateLocalTokenRanges(metadataFetcher, topology);
+ }
+
+ // todo: refactor to a utility class _when_ refactoring
TokenRangeReplicasResponse data structure (separate out server and http data
representations)
+ @NotNull
+ public static Map<Integer, Set<TokenRange>>
calculateLocalTokenRanges(InstanceMetadataFetcher metadataFetcher,
TokenRangeReplicasResponse topology)
+ {
+ if (topology == null)
+ {
+ return Collections.emptyMap();
Review Comment:
nit: Set.of
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RingTopologyRefresher.java:
##########
@@ -104,6 +153,86 @@ public TokenRangeReplicasResponse
cachedReplicaByTokenRange(RestoreJob restoreJo
return replicaByTokenRangePerKeyspace.forRestoreJob(restoreJob);
}
+ /**
+ * Fetch the latest topology view
+ * <p>It is synchronized when force refreshing as there is potential
contention from {@link #execute(Promise)}
+ *
+ * @param keyspace keyspace to determine replication
+ * @param forceRefresh whether refresh the topology view forcibly or not
+ * @return token ranges of the local Cassandra instances or an empty map
of nothing is found
+ */
+ @Override
+ public Map<Integer, Set<TokenRange>> localTokenRanges(String keyspace,
boolean forceRefresh)
+ {
+ TokenRangeReplicasResponse topology;
+ if (forceRefresh) // fetch the latest topology and load into cache
+ {
+ synchronized (this)
+ {
+ topology = prepareAndFetch((storageOperations, nodeSettings)
-> {
+ String partitioner = nodeSettings.partitioner();
+ return replicaByTokenRangePerKeyspace.loadOne(keyspace, k
-> storageOperations.tokenRangeReplicas(new Name(keyspace), partitioner));
+ });
+ }
+ }
+ else // get the cached value
+ {
+ topology =
replicaByTokenRangePerKeyspace.topologyOfKeyspace(keyspace);
+ }
+
+ return calculateLocalTokenRanges(metadataFetcher, topology);
+ }
+
+ // todo: refactor to a utility class _when_ refactoring
TokenRangeReplicasResponse data structure (separate out server and http data
representations)
+ @NotNull
+ public static Map<Integer, Set<TokenRange>>
calculateLocalTokenRanges(InstanceMetadataFetcher metadataFetcher,
TokenRangeReplicasResponse topology)
+ {
+ if (topology == null)
+ {
+ return Collections.emptyMap();
+ }
+
+ // todo: this assumes one C* node per IP address
+ Map<String, Integer> allNodes =
topology.replicaMetadata().values().stream()
+
.collect(Collectors.toMap(TokenRangeReplicasResponse.ReplicaMetadata::address,
+
TokenRangeReplicasResponse.ReplicaMetadata::port));
+
+ List<InstanceMetadata> localNodes =
metadataFetcher.allLocalInstances();
+ Map<String, InstanceMetadata> localEndpointsToMetadata = new
HashMap<>(localNodes.size());
+ for (InstanceMetadata instanceMetadata : localNodes)
+ {
+ populateEndpointToMetadata(instanceMetadata, allNodes,
localEndpointsToMetadata);
+ }
+
+ Map<Integer, Set<TokenRange>> localTokenRanges = new
HashMap<>(localEndpointsToMetadata.size());
+ for (TokenRangeReplicasResponse.ReplicaInfo ri :
topology.writeReplicas())
+ {
+ TokenRange range = new TokenRange(Token.from(ri.start()),
Token.from(ri.end()));
+ for (List<String> instanceOfDc :
ri.replicasByDatacenter().values())
+ {
+ for (String instanceEndpoint : instanceOfDc)
+ {
+ // skip the non-local nodes
+ if
(!localEndpointsToMetadata.containsKey(instanceEndpoint))
+ {
+ continue;
+ }
+
+ InstanceMetadata instanceMetadata =
localEndpointsToMetadata.get(instanceEndpoint);
+ localTokenRanges.compute(instanceMetadata.id(), (key,
value) -> {
+ if (value == null)
+ {
+ value = new HashSet<>();
+ }
+ value.add(range);
+ return value;
+ });
Review Comment:
I think this reads better:
```suggestion
InstanceMetadata instanceMetadata =
localEndpointsToMetadata.get(instanceEndpoint);
if (instanceMetadata == null)
{
continue;
}
localTokenRanges.computeIfAbsent(instanceMetadata.id(),
k -> new HashSet<>())
.add(range);
```
##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/request/data/CreateSliceRequestPayload.java:
##########
@@ -50,8 +50,8 @@ public class CreateSliceRequestPayload
private final String bucket;
private final String key;
private final String checksum;
- private final BigInteger startToken;
- private final BigInteger endToken;
+ private final BigInteger firstToken; // inclusive
Review Comment:
ok so the change from start token to first token is to differentiate that
this is not the start token. This makes sense
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RingTopologyRefresher.java:
##########
@@ -139,18 +268,65 @@ private <T> T
prepareAndFetch(BiFunction<StorageOperations, NodeSettings, T> fet
});
}
+ private void dispatchRingTopologyChangeAsync(String keyspace,
TokenRangeReplicasResponse oldTopology, TokenRangeReplicasResponse newTopology)
+ {
+ // Dispatch onRingTopologyChanged in another thread as it might block
+ // This method cannot block, since it is invoked by
org.apache.cassandra.sidecar.restore.RingTopologyRefresher.ReplicaByTokenRangePerKeyspace.loadOne
+ // in the `compute` lambda.
+ listenersByKeyspace.getOrDefault(keyspace,
Collections.emptySet()).forEach(listener -> {
+ executorPool.runBlocking(() ->
listener.onRingTopologyChanged(keyspace, oldTopology, newTopology));
+ });
+ }
+
+ private static void populateEndpointToMetadata(InstanceMetadata
instanceMetadata,
+ Map<String, Integer>
allNodes,
+ Map<String,
InstanceMetadata> localEndpointsToMetadata)
+ {
+ String ip = ipOf(instanceMetadata);
+ Integer port = allNodes.get(ip);
+ if (port == null)
+ {
+ // the configured local instance is not in the C* ring yet (not
even joining). Exit early for such node
+ return;
+ }
+ String endpointWithPort =
StringUtils.cassandraFormattedHostAndPort(ip, port);
+ localEndpointsToMetadata.put(endpointWithPort, instanceMetadata);
+ }
+
+ private static String ipOf(InstanceMetadata instanceMetadata)
+ {
+ String ipAddress = instanceMetadata.ipAddress();
+ if (ipAddress == null)
+ {
+ try
+ {
+ return instanceMetadata.refreshIpAddress();
+ }
+ catch (UnknownHostException uhe)
+ {
+ LOGGER.debug("Failed to resolve IP address. host={}",
instanceMetadata.host());
+ }
+ }
+ return ipAddress;
Review Comment:
this can be null, but we are not handling null from the call sites
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRange.java:
##########
@@ -116,6 +156,14 @@ public Token end()
return range.upperEndpoint();
}
+ /**
+ * @return end token as {@link BigInteger}. It is the last token enclosed
in the range.
+ */
+ public BigInteger endAsBigInt()
Review Comment:
there are a few places where there are calls to
`range.upperEndpoint().toBigInteger()` do we want to update usage in call sites
to use this method instead?
##########
server/src/main/java/org/apache/cassandra/sidecar/db/RestoreSlice.java:
##########
@@ -118,69 +107,40 @@ public boolean equals(Object obj)
}
/**
- * Splits the slice based on the topology changes.
- * <p></p>
- * When the current topology has pending ranges, it results into the slice
S being split into
- * multiple sub-ranges, say, s1 and s2. The updater splits, and persists
the new slices to database.
- * The original slice S is then deleted.
- * The new slices still reference to the same s3 object, i.e. {@code
<bucket/key/checksum>}
+ * Trim the slice based on the reference local token range.
+ *
+ * <p>The range of the slice might not be entirely enclosed by the
localTokenRange.
+ * In such case, the slice is trimmed to align with the localTokenRange.
For example,
+ * the slice covers range {@code (1, 100])} and the localTokenRange covers
{@code (50, 90]}.
+ * The slice is trimmed to match with the localTokenRange, updating the
range to {@code (50, 90]}.
+ * <p>The trimmed slice still reference to the same s3 object, i.e. {@code
<bucket/key/checksum>}
*
- * @param topology topology to guide the splitting
- * @return a list of slices of the sub-ranges
+ * @param localTokenRange local token range
+ * @return a restore slice that might be trimmed
*/
- public List<RestoreSlice> splitMaybe(@Nullable TokenRangeReplicasResponse
topology)
+ public RestoreSlice trimMaybe(@NotNull TokenRange localTokenRange)
{
- if (topology == null)
- {
- return Collections.singletonList(this);
- }
-
- List<RestoreSlice> splits = new ArrayList<>();
TokenRange sliceRange = new TokenRange(startToken(), endToken());
- for (TokenRangeReplicasResponse.ReplicaInfo replicaInfo :
topology.writeReplicas())
+ if (localTokenRange.encloses(sliceRange))
{
- TokenRange range = new TokenRange(Token.from(replicaInfo.start()),
Token.from(replicaInfo.end()));
- if (range.encloses(sliceRange))
- {
- return Collections.singletonList(this);
- }
- else if (range.overlaps(sliceRange))
- {
- TokenRange intersection = range.intersection(sliceRange);
- // Adjust the slice range to match with the write replica and
persist.
- // The object location remains the same as sidecar need to
download the same object.
- // It just needs a narrower range of data within the slice
- splits.add(unbuild()
- .startToken(intersection.start().toBigInteger())
- .endToken(intersection.end().toBigInteger())
- .build());
- }
-
- if (range.largerThan(sliceRange))
- {
- // all following ranges are larger the original range; exit
the iteration early.
- break;
- }
+ return this;
}
- if (splits.isEmpty())
- {
- throw new IllegalStateException("Token range of the slice is not
found in the write replicas. " +
- "slice range: " +
formatRange(startToken, endToken));
- }
- else
+ if (localTokenRange.overlaps(sliceRange))
Review Comment:
maybe this method should be called `intersects` instead?
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRange.java:
##########
@@ -33,7 +39,10 @@
*/
public class TokenRange
{
- private final Range<Token> range;
+ public static final Comparator<TokenRange> NATURAL_ORDER =
Comparator.comparing(TokenRange::start)
Review Comment:
remove, also `firstToken` is only used by tests?
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RingTopologyRefresher.java:
##########
@@ -241,6 +431,7 @@ void loadEach(String keyspace, Function<String,
TokenRangeReplicasResponse> load
promise.tryComplete(topology);
return promise;
});
+ asyncDispatcher.onRingTopologyChanged(keyspace, null,
topology);
Review Comment:
i'm generally concerned about dispatching here. can we do it outside of the
compute block? I think the async nature gives us some guarantees, but I'm still
worried about having this here.
`shouldDispatch.set(true);`
...
`if (shoudDispatch.get()) asyncDispatch`
##########
server/src/main/java/org/apache/cassandra/sidecar/tasks/PeriodicTaskExecutor.java:
##########
@@ -91,7 +91,7 @@ private void schedule(PeriodicTaskKey key, long
priorExecDurationMillis, long de
return v;
}
- LOGGER.debug("Scheduling task {}. task='{}' execCount={}",
+ LOGGER.trace("Scheduling task {}. task='{}' execCount={}",
Review Comment:
BIG -1 on changing debug to trace. I use debug on this class frequently.
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RestoreJobConsistencyChecker.java:
##########
@@ -146,45 +150,103 @@ private boolean
shouldForceRestoreJobDiscoverRun(RestoreJob restoreJob, List<Res
return false;
}
+ @SuppressWarnings("UnstableApiUsage")
private static void concludeRanges(List<RestoreRange> ranges,
TokenRangeReplicasResponse topology,
ConsistencyVerifier verifier,
RestoreRangeStatus successCriteria,
RestoreJobProgressCollector collector)
{
- for (RestoreRange range : ranges)
+ RangeMap<Token, InstanceSetByDc> replicasPerRange =
populateReplicas(topology);
+ Map<Range<Token>, Pair<Map<String, RestoreRangeStatus>, RestoreRange>>
statusPerRange = populateStatusByReplica(ranges);
+
+ for (Map.Entry<Range<Token>, Pair<Map<String, RestoreRangeStatus>,
RestoreRange>> entry : statusPerRange.entrySet())
{
if (!collector.canCollectMore())
{
return;
}
- ConsistencyVerificationResult res = concludeOneRange(topology,
verifier, successCriteria, range);
- collector.collect(range, res);
+ Range<Token> tokenRange = entry.getKey();
+ Map<String, RestoreRangeStatus> status =
entry.getValue().getLeft();
+ RestoreRange relevantRestoreRange = entry.getValue().getRight();
+ ConsistencyVerificationResult res =
concludeOneRange(replicasPerRange, verifier, successCriteria, tokenRange,
status);
+ collector.collect(relevantRestoreRange, res);
}
}
+ @SuppressWarnings("UnstableApiUsage")
+ static RangeMap<Token, InstanceSetByDc>
populateReplicas(TokenRangeReplicasResponse topology)
+ {
+ RangeMap<Token, InstanceSetByDc> replicasPerRange =
TreeRangeMap.create();
+ for (TokenRangeReplicasResponse.ReplicaInfo replicaInfo :
topology.writeReplicas())
+ {
+ TokenRange tokenRange = new
TokenRange(Token.from(replicaInfo.start()), Token.from(replicaInfo.end()));
+ Map<String, List<String>> replicasByDc =
replicaInfo.replicasByDatacenter();
+ Map<String, Set<String>> mapping = new
HashMap<>(replicasByDc.size());
+ replicasByDc.forEach((k, instances) -> mapping.put(k, new
HashSet<>(instances)));
Review Comment:
```suggestion
replicasByDc.forEach((dc, instances) -> mapping.put(dc, new
HashSet<>(instances)));
```
##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RingTopologyRefresher.java:
##########
@@ -160,35 +336,48 @@ boolean isEmpty()
// returns keyspace name
String register(RestoreJob restoreJob)
{
- allJobs.add(restoreJob.jobId);
+ // the job is already registered
+ if (!allJobs.add(restoreJob.jobId))
+ {
+ return restoreJob.keyspaceName;
+ }
+
jobsByKeyspace.compute(restoreJob.keyspaceName, (ks, jobs) -> {
- Set<UUID> jobsSet = jobs == null ? new HashSet<>() : jobs;
+ Set<UUID> jobsSet = jobs == null ?
ConcurrentHashMap.newKeySet() : jobs;
jobsSet.add(restoreJob.jobId);
return jobsSet;
});
return restoreJob.keyspaceName;
Review Comment:
I think this simplifies things a bit:
```
// the job is not yet registered
if (allJobs.add(restoreJob.jobId))
{
jobsByKeyspace.computeIfAbsent(restoreJob.keyspaceName, k ->
ConcurrentHashMap.newKeySet())
.add(restoreJob.jobId);
}
return restoreJob.keyspaceName;
```
##########
server-common/src/test/java/org/apache/cassandra/sidecar/common/server/cluster/locator/TokenRangeTest.java:
##########
@@ -159,6 +165,66 @@ void testIntersection()
assertThat(r2.intersection(r4)).isNotEqualTo(new TokenRange(5, 5)); //
but not any empty range
}
+ @ParameterizedTest(name = "{index} - {0}: inputLeft={1} inputRight={2}
expectedLeft={3} expectedRight={4}")
+ @MethodSource("inputAndExpectedResultAfterDiff")
+ void testDiff(String testTitle, Set<TokenRange> left, Set<TokenRange>
right, Set<TokenRange> expectedLeft, Set<TokenRange> expectedRight)
+ {
+ TokenRange.Pair diff = TokenRange.diff(left, right);
+ assertThat(diff.left).isEqualTo(expectedLeft);
+ assertThat(diff.right).isEqualTo(expectedRight);
+
+ // exchange left and right; it is to test the commutative property of
diff
+ diff = TokenRange.diff(right, left);
+ assertThat(diff.left).isEqualTo(expectedRight);
+ assertThat(diff.right).isEqualTo(expectedLeft);
+ }
+
+ public static Stream<Arguments> inputAndExpectedResultAfterDiff()
+ {
+ return Stream.of(
+ // inputLeft, inputRight, expectedLeft, expectedRight
+ args("Diff on identical sets",
+ ImmutableSet.of(r(0, 1000), r(1000, 2000)), // inputLeft
Review Comment:
NIT: Use `Set.of` instead since now we are using JDK11 APIs
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/exceptions/SchemaUnavailableException.java:
##########
@@ -16,21 +16,21 @@
* limitations under the License.
*/
-package org.apache.cassandra.sidecar.exceptions;
+package org.apache.cassandra.sidecar.common.server.exceptions;
Review Comment:
What I meant is that in the commons subproject, we should still use the
`org.apache.cassandra.sidecar.exceptions` package
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]