maedhroz commented on a change in pull request #1146: URL: https://github.com/apache/cassandra/pull/1146#discussion_r690680751
########## File path: test/distributed/org/apache/cassandra/distributed/test/UnableToParseClientMessageFromBlockedSubnetTest.java ########## @@ -0,0 +1,134 @@ +/* + * 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.distributed.test; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeoutException; + +import com.google.common.collect.ImmutableMap; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.transport.Message; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.transport.SimpleClient; +import org.apache.cassandra.transport.messages.ErrorMessage; +import org.assertj.core.api.Assertions; + +@RunWith(Parameterized.class) +public class UnableToParseClientMessageFromBlockedSubnetTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + private static List<String> CLUSTER_EXCLUDED_SUBNETS; + + @Parameterized.Parameter(0) + public List<String> excludeSubnets; + @Parameterized.Parameter(1) + public ProtocolVersion version; + + @Parameterized.Parameters(name = "domains={0},version={1}") + public static Iterable<Object[]> params() + { + List<Object[]> tests = new ArrayList<>(); + for (List<String> domains : Arrays.asList(Arrays.asList("127.0.0.1"), Arrays.asList("127.0.0.0/31"))) Review comment: nit: `Collections.singletonList()` is a little less memory hungry, but this is a test, so not a big deal. ########## File path: test/distributed/org/apache/cassandra/distributed/test/UnableToParseClientMessageTest.java ########## @@ -20,11 +20,15 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Predicate; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Uninterruptibles; Review comment: nit: The 4 imports above look unused. ########## File path: src/java/org/apache/cassandra/config/Config.java ########## @@ -568,6 +568,8 @@ public static void setClientMode(boolean clientMode) public volatile int consecutive_message_errors_threshold = 1; + public volatile SubnetGroups client_error_reporting_exclusions = new SubnetGroups(); Review comment: Do we need to add `SubnetGroups` to `DatabaseDescriptorRefTest`? ########## File path: src/java/org/apache/cassandra/config/SubnetGroups.java ########## @@ -0,0 +1,107 @@ +/* + * 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.config; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import inet.ipaddr.IPAddressNetwork; +import inet.ipaddr.IPAddressString; + +/** + * When a group of subnets are needed, this class can be used to represent the group as if it was a single subnet. + * + * This class supports IPV4 and IPV6 subnets + */ +public class SubnetGroups +{ + public Set<Group> subnets = Collections.emptySet(); + + public SubnetGroups() + { + } + + public SubnetGroups(List<String> values) Review comment: nit: Could throw a `@SuppressWarnings("unused")` on here if this is the constructor snakeyaml uses? ########## File path: src/java/org/apache/cassandra/config/SubnetGroups.java ########## @@ -0,0 +1,107 @@ +/* + * 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.config; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import inet.ipaddr.IPAddressNetwork; +import inet.ipaddr.IPAddressString; + +/** + * When a group of subnets are needed, this class can be used to represent the group as if it was a single subnet. + * + * This class supports IPV4 and IPV6 subnets + */ +public class SubnetGroups +{ + public Set<Group> subnets = Collections.emptySet(); + + public SubnetGroups() + { + } + + public SubnetGroups(List<String> values) + { + this.subnets = ImmutableSet.copyOf(values.stream().map(Group::new).collect(Collectors.toSet())); + } + + public boolean contains(SocketAddress address) + { Review comment: nit: Alternate method body idea: ``` public boolean contains(SocketAddress address) { Preconditions.checkNotNull(address); Preconditions.checkArgument(address instanceof InetSocketAddress, "Unsupported socket address type: " + address.getClass()); return contains(((InetSocketAddress) address).getAddress()); } ``` ########## File path: src/java/org/apache/cassandra/config/SubnetGroups.java ########## @@ -0,0 +1,107 @@ +/* + * 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.config; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import inet.ipaddr.IPAddressNetwork; +import inet.ipaddr.IPAddressString; + +/** + * When a group of subnets are needed, this class can be used to represent the group as if it was a single subnet. + * + * This class supports IPV4 and IPV6 subnets + */ +public class SubnetGroups +{ + public Set<Group> subnets = Collections.emptySet(); + + public SubnetGroups() + { + } + + public SubnetGroups(List<String> values) + { + this.subnets = ImmutableSet.copyOf(values.stream().map(Group::new).collect(Collectors.toSet())); + } + + public boolean contains(SocketAddress address) + { + if (address instanceof InetSocketAddress) + { + return contains(((InetSocketAddress) address).getAddress()); + } + throw new IllegalArgumentException("Unsupported socket address type: " + (address == null ? null : address.getClass())); + } + + public boolean contains(InetAddress address) + { + for (Group group : subnets) + { + if (group.contains(address)) + { + return true; + } + } + return false; + } + + public static class Group Review comment: nit: `Group` could be private? ########## File path: src/java/org/apache/cassandra/config/SubnetGroups.java ########## @@ -0,0 +1,107 @@ +/* + * 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.config; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import inet.ipaddr.IPAddressNetwork; +import inet.ipaddr.IPAddressString; + +/** + * When a group of subnets are needed, this class can be used to represent the group as if it was a single subnet. + * + * This class supports IPV4 and IPV6 subnets + */ +public class SubnetGroups +{ + public Set<Group> subnets = Collections.emptySet(); + + public SubnetGroups() + { + } + + public SubnetGroups(List<String> values) + { + this.subnets = ImmutableSet.copyOf(values.stream().map(Group::new).collect(Collectors.toSet())); + } + + public boolean contains(SocketAddress address) + { + if (address instanceof InetSocketAddress) + { + return contains(((InetSocketAddress) address).getAddress()); + } + throw new IllegalArgumentException("Unsupported socket address type: " + (address == null ? null : address.getClass())); + } + + public boolean contains(InetAddress address) + { + for (Group group : subnets) + { + if (group.contains(address)) + { + return true; + } + } + return false; + } + + public static class Group + { + private static final IPAddressNetwork.IPAddressGenerator IP_ADDRESS_GENERATOR = new IPAddressNetwork.IPAddressGenerator(); + + private final IPAddressString subnet; + + public Group(String range) + { + subnet = new IPAddressString(range); + } + + public boolean contains(InetAddress address) + { + return subnet.contains(IP_ADDRESS_GENERATOR.from(address).toAddressString()); + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Group group = (Group) o; + return Objects.equals(subnet, group.subnet); Review comment: ```suggestion return subnet.equals(group.subnet); ``` I think this would avoid a redundancy with the reference equality test. ########## File path: src/java/org/apache/cassandra/config/SubnetGroups.java ########## @@ -0,0 +1,107 @@ +/* + * 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.config; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import inet.ipaddr.IPAddressNetwork; +import inet.ipaddr.IPAddressString; + +/** + * When a group of subnets are needed, this class can be used to represent the group as if it was a single subnet. + * + * This class supports IPV4 and IPV6 subnets + */ +public class SubnetGroups +{ + public Set<Group> subnets = Collections.emptySet(); + + public SubnetGroups() + { + } + + public SubnetGroups(List<String> values) + { + this.subnets = ImmutableSet.copyOf(values.stream().map(Group::new).collect(Collectors.toSet())); + } + + public boolean contains(SocketAddress address) + { + if (address instanceof InetSocketAddress) + { + return contains(((InetSocketAddress) address).getAddress()); + } + throw new IllegalArgumentException("Unsupported socket address type: " + (address == null ? null : address.getClass())); + } + + public boolean contains(InetAddress address) + { + for (Group group : subnets) + { + if (group.contains(address)) + { + return true; + } + } + return false; + } + + public static class Group + { + private static final IPAddressNetwork.IPAddressGenerator IP_ADDRESS_GENERATOR = new IPAddressNetwork.IPAddressGenerator(); + + private final IPAddressString subnet; + + public Group(String range) + { + subnet = new IPAddressString(range); + } + + public boolean contains(InetAddress address) + { + return subnet.contains(IP_ADDRESS_GENERATOR.from(address).toAddressString()); + } + + public boolean equals(Object o) Review comment: nit: missing `@Override` annotations on `equals()`, `hashCode()`, and `toString()` ########## File path: src/java/org/apache/cassandra/config/Config.java ########## @@ -568,6 +568,8 @@ public static void setClientMode(boolean clientMode) public volatile int consecutive_message_errors_threshold = 1; + public volatile SubnetGroups client_error_reporting_exclusions = new SubnetGroups(); Review comment: Aside: Probably a good idea to make this `volatile` for now even if we don't actually allow modifying it at runtime. ########## File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java ########## @@ -3406,4 +3406,9 @@ public static void setConsecutiveMessageErrorsThreshold(int value) { conf.consecutive_message_errors_threshold = value; } + + public static SubnetGroups getExcludeClientErrorsFrom() Review comment: nit: Oddly, it feels like this would read better as... ``` if (DatabaseDescriptor.getClientErrorReportingExclusions().contains(ctx.channel().remoteAddress())) ``` i.e. "the exclusion set contains" in prose ########## File path: src/java/org/apache/cassandra/transport/PreV5Handlers.java ########## @@ -244,6 +245,14 @@ public void exceptionCaught(final ChannelHandlerContext ctx, Throwable cause) if (isFatal(cause)) future.addListener((ChannelFutureListener) f -> ctx.close()); } + if (DatabaseDescriptor.getExcludeClientErrorsFrom().contains(ctx.channel().remoteAddress())) + { + // some times it is desirable to ignore exceptions from specific IPs; such as when security scans are Review comment: ```suggestion // Sometimes it is desirable to ignore exceptions from specific IPs; such as when security scans are ``` ########## File path: src/java/org/apache/cassandra/transport/PreV5Handlers.java ########## @@ -244,6 +245,14 @@ public void exceptionCaught(final ChannelHandlerContext ctx, Throwable cause) if (isFatal(cause)) future.addListener((ChannelFutureListener) f -> ctx.close()); } + if (DatabaseDescriptor.getExcludeClientErrorsFrom().contains(ctx.channel().remoteAddress())) + { + // some times it is desirable to ignore exceptions from specific IPs; such as when security scans are + // running. To avoid polluting logs and metrics, metrics are not updated when the IP is in the exclude + // list Review comment: ```suggestion // list. ``` -- 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]

