-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
3 changed files
with
145 additions
and
77 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
57 changes: 57 additions & 0 deletions
57
src/java/org/apache/cassandra/service/reads/trackwarnings/WarnAbortCounter.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,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.service.reads.trackwarnings; | ||
|
||
import java.util.Collections; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import org.apache.cassandra.locator.InetAddressAndPort; | ||
|
||
public class WarnAbortCounter | ||
{ | ||
final Set<InetAddressAndPort> warnings = Collections.newSetFromMap(new ConcurrentHashMap<>()); | ||
// the highest number reported by a node's warning | ||
final AtomicLong maxWarningValue = new AtomicLong(); | ||
|
||
final Set<InetAddressAndPort> aborts = Collections.newSetFromMap(new ConcurrentHashMap<>()); | ||
// the highest number reported by a node's rejection. | ||
final AtomicLong maxAbortsValue = new AtomicLong(); | ||
|
||
void addWarning(InetAddressAndPort from, long value) | ||
{ | ||
maxWarningValue.accumulateAndGet(value, Math::max); | ||
// call add last so concurrent reads see empty even if values > 0; if done in different order then | ||
// size=1 could have values == 0 | ||
warnings.add(from); | ||
} | ||
|
||
void addAbort(InetAddressAndPort from, long value) | ||
{ | ||
maxAbortsValue.accumulateAndGet(value, Math::max); | ||
// call add last so concurrent reads see empty even if values > 0; if done in different order then | ||
// size=1 could have values == 0 | ||
aborts.add(from); | ||
} | ||
|
||
public WarningsSnapshot.Warnings snapshot() | ||
{ | ||
return WarningsSnapshot.Warnings.create(WarningsSnapshot.Counter.create(warnings, maxWarningValue), WarningsSnapshot.Counter.create(aborts, maxAbortsValue)); | ||
} | ||
} |
83 changes: 83 additions & 0 deletions
83
src/java/org/apache/cassandra/service/reads/trackwarnings/WarningContext.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,83 @@ | ||
/* | ||
* 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.service.reads.trackwarnings; | ||
|
||
import java.util.Collections; | ||
import java.util.EnumSet; | ||
import java.util.Map; | ||
import java.util.Set; | ||
|
||
import org.apache.cassandra.exceptions.RequestFailureReason; | ||
import org.apache.cassandra.locator.InetAddressAndPort; | ||
import org.apache.cassandra.net.ParamType; | ||
|
||
public class WarningContext | ||
{ | ||
private static EnumSet<ParamType> SUPPORTED = EnumSet.of(ParamType.TOMBSTONE_WARNING, ParamType.TOMBSTONE_ABORT, | ||
ParamType.LOCAL_READ_SIZE_WARN, ParamType.LOCAL_READ_SIZE_ABORT, | ||
ParamType.ROW_INDEX_SIZE_WARN, ParamType.ROW_INDEX_SIZE_ABORT); | ||
|
||
final WarnAbortCounter tombstones = new WarnAbortCounter(); | ||
final WarnAbortCounter localReadSize = new WarnAbortCounter(); | ||
final WarnAbortCounter rowIndexTooLarge = new WarnAbortCounter(); | ||
|
||
public static boolean isSupported(Set<ParamType> keys) | ||
{ | ||
return !Collections.disjoint(keys, SUPPORTED); | ||
} | ||
|
||
public RequestFailureReason updateCounters(Map<ParamType, Object> params, InetAddressAndPort from) | ||
{ | ||
for (Map.Entry<ParamType, Object> entry : params.entrySet()) | ||
{ | ||
WarnAbortCounter counter = null; | ||
RequestFailureReason reason = null; | ||
switch (entry.getKey()) | ||
{ | ||
case ROW_INDEX_SIZE_ABORT: | ||
reason = RequestFailureReason.READ_SIZE; | ||
case ROW_INDEX_SIZE_WARN: | ||
counter = rowIndexTooLarge; | ||
break; | ||
case LOCAL_READ_SIZE_ABORT: | ||
reason = RequestFailureReason.READ_SIZE; | ||
case LOCAL_READ_SIZE_WARN: | ||
counter = localReadSize; | ||
break; | ||
case TOMBSTONE_ABORT: | ||
reason = RequestFailureReason.READ_TOO_MANY_TOMBSTONES; | ||
case TOMBSTONE_WARNING: | ||
counter = tombstones; | ||
break; | ||
} | ||
if (reason != null) | ||
{ | ||
counter.addAbort(from, ((Number) entry.getValue()).longValue()); | ||
return reason; | ||
} | ||
if (counter != null) | ||
counter.addWarning(from, ((Number) entry.getValue()).longValue()); | ||
} | ||
return null; | ||
} | ||
|
||
public WarningsSnapshot snapshot() | ||
{ | ||
return WarningsSnapshot.create(tombstones.snapshot(), localReadSize.snapshot(), rowIndexTooLarge.snapshot()); | ||
} | ||
} |