HBASE-26293 Use reservoir sampling when selecting bootstrap nodes (#3702)

Signed-off-by: Xiaolin Ha <haxiaolin@apache.org>
This commit is contained in:
Duo Zhang 2021-09-28 23:27:47 +08:00 committed by GitHub
parent 748a6827c4
commit 113cc2a591
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 177 additions and 12 deletions

View File

@ -0,0 +1,72 @@
/**
* 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.hadoop.hbase.util;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Stream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
* The simple version of reservoir sampling implementation. It is enough for the usage in HBase.
* <p/>
* See https://en.wikipedia.org/wiki/Reservoir_sampling.
*/
@InterfaceAudience.Private
public class ReservoirSample<T> {
private final List<T> r;
private final int k;
private int n;
public ReservoirSample(int k) {
Preconditions.checkArgument(k > 0, "negative sampling number(%d) is not allowed");
r = new ArrayList<>(k);
this.k = k;
}
public void add(T t) {
if (n < k) {
r.add(t);
} else {
int j = ThreadLocalRandom.current().nextInt(n + 1);
if (j < k) {
r.set(j, t);
}
}
n++;
}
public void add(Iterator<T> iter) {
iter.forEachRemaining(this::add);
}
public void add(Stream<T> s) {
s.forEachOrdered(this::add);
}
public List<T> getSamplingResult() {
return r;
}
}

View File

@ -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.hadoop.hbase.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.stream.IntStream;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MiscTests.class, SmallTests.class })
public class TestReservoirSample {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestReservoirSample.class);
@Test
public void test() {
int round = 100000;
int containsOne = 0;
for (int i = 0; i < round; i++) {
ReservoirSample<Integer> rs = new ReservoirSample<>(10);
for (int j = 0; j < 100; j++) {
rs.add(j);
if (j < 10) {
assertEquals(j + 1, rs.getSamplingResult().size());
} else {
assertEquals(10, rs.getSamplingResult().size());
}
}
if (rs.getSamplingResult().contains(1)) {
containsOne++;
}
}
// we assume a 5% error rate
assertTrue(containsOne > round / 10 * 0.95);
assertTrue(containsOne < round / 10 * 1.05);
}
@Test
public void testIterator() {
int round = 100000;
int containsOne = 0;
for (int i = 0; i < round; i++) {
ReservoirSample<Integer> rs = new ReservoirSample<>(10);
rs.add(IntStream.range(0, 100).mapToObj(Integer::valueOf).iterator());
if (rs.getSamplingResult().contains(1)) {
containsOne++;
}
}
// we assume a 5% error rate
assertTrue(containsOne > round / 10 * 0.95);
assertTrue(containsOne < round / 10 * 1.05);
}
@Test
public void testStream() {
int round = 100000;
int containsOne = 0;
for (int i = 0; i < round; i++) {
ReservoirSample<Integer> rs = new ReservoirSample<>(10);
rs.add(IntStream.range(0, 100).mapToObj(Integer::valueOf));
if (rs.getSamplingResult().contains(1)) {
containsOne++;
}
}
// we assume a 5% error rate
assertTrue(containsOne > round / 10 * 0.95);
assertTrue(containsOne < round / 10 * 1.05);
}
}

View File

@ -22,11 +22,9 @@ import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.BindException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
@ -53,6 +51,7 @@ import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.OOMEChecker;
import org.apache.hadoop.hbase.util.ReservoirSample;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -299,12 +298,13 @@ public abstract class HBaseRpcServicesBase<S extends HBaseServerBase<?>>
@Override
public final GetBootstrapNodesResponse getBootstrapNodes(RpcController controller,
GetBootstrapNodesRequest request) throws ServiceException {
List<ServerName> bootstrapNodes = new ArrayList<>(server.getRegionServers());
Collections.shuffle(bootstrapNodes, ThreadLocalRandom.current());
int maxNodeCount = server.getConfiguration().getInt(CLIENT_BOOTSTRAP_NODE_LIMIT,
DEFAULT_CLIENT_BOOTSTRAP_NODE_LIMIT);
ReservoirSample<ServerName> sample = new ReservoirSample<>(maxNodeCount);
sample.add(server.getRegionServers());
GetBootstrapNodesResponse.Builder builder = GetBootstrapNodesResponse.newBuilder();
bootstrapNodes.stream().limit(maxNodeCount).map(ProtobufUtil::toServerName)
sample.getSamplingResult().stream().map(ProtobufUtil::toServerName)
.forEach(builder::addServerName);
return builder.build();
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import org.apache.hadoop.hbase.HRegionLocation;
@ -46,9 +46,9 @@ public interface ConnectionRegistryEndpoint {
List<ServerName> getBackupMasters();
/**
* Get all the region servers address.
* Get a iterator of the region servers which could be used as bootstrap nodes.
*/
Collection<ServerName> getRegionServers();
Iterator<ServerName> getRegionServers();
/**
* Get the location of meta regions.

View File

@ -4003,8 +4003,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
}
@Override
public Collection<ServerName> getRegionServers() {
return regionServerTracker.getRegionServers();
public Iterator<ServerName> getRegionServers() {
return regionServerTracker.getRegionServers().iterator();
}
@Override

View File

@ -34,6 +34,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -3429,8 +3430,8 @@ public class HRegionServer extends HBaseServerBase<RSRpcServices>
}
@Override
public Collection<ServerName> getRegionServers() {
return regionServerAddressTracker.getRegionServers();
public Iterator<ServerName> getRegionServers() {
return regionServerAddressTracker.getRegionServers().iterator();
}
@Override