Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Rackaware placement policy support local node awareness by hostname #4057

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -58,20 +58,40 @@ protected RackawareEnsemblePlacementPolicy initialize(DNSToSwitchMapping dnsReso
boolean enforceMinNumRacksPerWriteQuorum,
boolean ignoreLocalNodeInPlacementPolicy,
StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) {
return initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds,
reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum,
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, false,
statsLogger, bookieAddressResolver);
}

@Override
protected RackawareEnsemblePlacementPolicy initialize(DNSToSwitchMapping dnsResolver,
HashedWheelTimer timer,
boolean reorderReadsRandom,
int stabilizePeriodSeconds,
int reorderThresholdPendingRequests,
boolean isWeighted,
int maxWeightMultiple,
int minNumRacksPerWriteQuorum,
boolean enforceMinNumRacksPerWriteQuorum,
boolean ignoreLocalNodeInPlacementPolicy,
boolean useHostnameResolveLocalNodePlacementPolicy,
hangc0276 marked this conversation as resolved.
Show resolved Hide resolved
StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) {
if (stabilizePeriodSeconds > 0) {
super.initialize(dnsResolver, timer, reorderReadsRandom, 0, reorderThresholdPendingRequests, isWeighted,
maxWeightMultiple, minNumRacksPerWriteQuorum, enforceMinNumRacksPerWriteQuorum,
ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver);
ignoreLocalNodeInPlacementPolicy, useHostnameResolveLocalNodePlacementPolicy,
statsLogger, bookieAddressResolver);
slave = new RackawareEnsemblePlacementPolicyImpl(enforceDurability);
slave.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds,
reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum,
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger,
bookieAddressResolver);
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy,
useHostnameResolveLocalNodePlacementPolicy, statsLogger, bookieAddressResolver);
} else {
super.initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds,
reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum,
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy, statsLogger,
bookieAddressResolver);
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy,
useHostnameResolveLocalNodePlacementPolicy, statsLogger, bookieAddressResolver);
slave = null;
}
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
protected int minNumRacksPerWriteQuorum;
protected boolean enforceMinNumRacksPerWriteQuorum;
protected boolean ignoreLocalNodeInPlacementPolicy;
protected boolean useHostnameResolveLocalNodePlacementPolicy;

public static final String REPP_RANDOM_READ_REORDERING = "ensembleRandomReadReordering";

Expand Down Expand Up @@ -144,6 +145,41 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
topology = new NetworkTopologyImpl();
}

/**
* Initialize the policy.
*
* @param dnsResolver
* @param timer
* @param reorderReadsRandom
* @param stabilizePeriodSeconds
* @param reorderThresholdPendingRequests
* @param isWeighted
* @param maxWeightMultiple
* @param minNumRacksPerWriteQuorum
* @param enforceMinNumRacksPerWriteQuorum
* @param ignoreLocalNodeInPlacementPolicy
* @param statsLogger
* @param bookieAddressResolver
* @return initialized ensemble placement policy
*/
protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dnsResolver,
HashedWheelTimer timer,
boolean reorderReadsRandom,
int stabilizePeriodSeconds,
int reorderThresholdPendingRequests,
boolean isWeighted,
int maxWeightMultiple,
int minNumRacksPerWriteQuorum,
boolean enforceMinNumRacksPerWriteQuorum,
boolean ignoreLocalNodeInPlacementPolicy,
StatsLogger statsLogger,
BookieAddressResolver bookieAddressResolver) {
return initialize(dnsResolver, timer, reorderReadsRandom, stabilizePeriodSeconds,
reorderThresholdPendingRequests, isWeighted, maxWeightMultiple, minNumRacksPerWriteQuorum,
enforceMinNumRacksPerWriteQuorum, ignoreLocalNodeInPlacementPolicy,
false, statsLogger, bookieAddressResolver);
}

/**
* Initialize the policy.
*
Expand All @@ -160,6 +196,7 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns
int minNumRacksPerWriteQuorum,
boolean enforceMinNumRacksPerWriteQuorum,
boolean ignoreLocalNodeInPlacementPolicy,
boolean useHostnameResolveLocalNodePlacementPolicy,
hangc0276 marked this conversation as resolved.
Show resolved Hide resolved
StatsLogger statsLogger,
BookieAddressResolver bookieAddressResolver) {
checkNotNull(statsLogger, "statsLogger should not be null, use NullStatsLogger instead.");
Expand Down Expand Up @@ -195,6 +232,7 @@ public Integer getSample() {
this.minNumRacksPerWriteQuorum = minNumRacksPerWriteQuorum;
this.enforceMinNumRacksPerWriteQuorum = enforceMinNumRacksPerWriteQuorum;
this.ignoreLocalNodeInPlacementPolicy = ignoreLocalNodeInPlacementPolicy;
this.useHostnameResolveLocalNodePlacementPolicy = useHostnameResolveLocalNodePlacementPolicy;

// create the network topology
if (stabilizePeriodSeconds > 0) {
Expand All @@ -206,7 +244,9 @@ public Integer getSample() {
BookieNode bn = null;
if (!ignoreLocalNodeInPlacementPolicy) {
try {
bn = createDummyLocalBookieNode(InetAddress.getLocalHost().getHostAddress());
String hostname = useHostnameResolveLocalNodePlacementPolicy
? InetAddress.getLocalHost().getCanonicalHostName() : InetAddress.getLocalHost().getHostAddress();
bn = createDummyLocalBookieNode(hostname);
} catch (IOException e) {
LOG.error("Failed to get local host address : ", e);
}
Expand Down Expand Up @@ -303,6 +343,7 @@ public Long load(BookieId key) throws Exception {
conf.getMinNumRacksPerWriteQuorum(),
conf.getEnforceMinNumRacksPerWriteQuorum(),
conf.getIgnoreLocalNodeInPlacementPolicy(),
conf.getUseHostnameResolveLocalNodePlacementPolicy(),
statsLogger,
bookieAddressResolver);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,8 @@ public void handleBookiesThatJoined(Set<BookieId> joinedBookies) {
.initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds,
this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple,
this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum,
this.ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver)
this.ignoreLocalNodeInPlacementPolicy,
this.useHostnameResolveLocalNodePlacementPolicy, statsLogger, bookieAddressResolver)
.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK));
}

Expand Down Expand Up @@ -201,7 +202,8 @@ public void onBookieRackChange(List<BookieId> bookieAddressList) {
this.stabilizePeriodSeconds, this.reorderThresholdPendingRequests,
this.isWeighted, this.maxWeightMultiple,
this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum,
this.ignoreLocalNodeInPlacementPolicy, statsLogger,
this.ignoreLocalNodeInPlacementPolicy,
this.useHostnameResolveLocalNodePlacementPolicy, statsLogger,
bookieAddressResolver)
.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
perRegionPlacement.put(newRegion, newRegionPlacement);
Expand Down Expand Up @@ -242,7 +244,8 @@ public RegionAwareEnsemblePlacementPolicy initialize(ClientConfiguration conf,
.initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds,
this.reorderThresholdPendingRequests, this.isWeighted, this.maxWeightMultiple,
this.minNumRacksPerWriteQuorum, this.enforceMinNumRacksPerWriteQuorum,
this.ignoreLocalNodeInPlacementPolicy, statsLogger, bookieAddressResolver)
this.ignoreLocalNodeInPlacementPolicy, this.ignoreLocalNodeInPlacementPolicy,
statsLogger, bookieAddressResolver)
.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK));
}
minRegionsForDurability = conf.getInt(REPP_MINIMUM_REGIONS_FOR_DURABILITY,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,9 @@ public class ClientConfiguration extends AbstractConfiguration<ClientConfigurati
protected static final String ENSEMBLE_PLACEMENT_POLICY_ORDER_SLOW_BOOKIES =
"ensemblePlacementPolicyOrderSlowBookies";
protected static final String BOOKIE_ADDRESS_RESOLVER_ENABLED = "bookieAddressResolverEnabled";
// Use hostname to resolve local placement info
public static final String USE_HOSTNAME_RESOLVE_LOCAL_NODE_PLACEMENT_POLICY =
"useHostnameResolveLocalNodePlacementPolicy";

// Stats
protected static final String ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
Expand Down Expand Up @@ -1314,6 +1317,22 @@ public ClientConfiguration setBookieAddressResolverEnabled(boolean enabled) {
return this;
}

/**
* Set the flag to use hostname to resolve local node placement policy.
* @param useHostnameResolveLocalNodePlacementPolicy
*/
public void setUseHostnameResolveLocalNodePlacementPolicy(boolean useHostnameResolveLocalNodePlacementPolicy) {
setProperty(USE_HOSTNAME_RESOLVE_LOCAL_NODE_PLACEMENT_POLICY, useHostnameResolveLocalNodePlacementPolicy);
}

/**
* Get whether to use hostname to resolve local node placement policy.
* @return
*/
public boolean getUseHostnameResolveLocalNodePlacementPolicy() {
return getBoolean(USE_HOSTNAME_RESOLVE_LOCAL_NODE_PLACEMENT_POLICY, false);
}

/**
* Whether to enable recording task execution stats.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1689,6 +1689,90 @@ public void testNewEnsembleWithPickDifferentRack() throws Exception {
}
}

@Test
public void testNewEnsemblePickLocalRackBookiesByHostname() throws Exception {
hangc0276 marked this conversation as resolved.
Show resolved Hide resolved
testNewEnsemblePickLocalRackBookiesInternal(true);
}

@Test
public void testNewEnsemblePickLocalRackBookiesByIP() throws Exception {
testNewEnsemblePickLocalRackBookiesInternal(false);
}

public void testNewEnsemblePickLocalRackBookiesInternal(boolean useHostnameResolveLocalNodePlacementPolicy)
throws Exception {
BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.1", 3181);
BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181);
BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.3", 3181);
BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181);
BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.5", 3181);
BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.6", 3181);
BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.7", 3181);

// update dns mapping
StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/default-region/r1");
StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2");
StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2");
StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r2");
StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/default-region/r3");
StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/default-region/r4");
StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/default-region/r5");

String hostname = useHostnameResolveLocalNodePlacementPolicy
? InetAddress.getLocalHost().getCanonicalHostName() : InetAddress.getLocalHost().getHostAddress();
StaticDNSResolver.addNodeToRack(hostname, "/default-region/r1");
if (useHostnameResolveLocalNodePlacementPolicy) {
conf.setUseHostnameResolveLocalNodePlacementPolicy(useHostnameResolveLocalNodePlacementPolicy);
}

repp.initialize(conf, Optional.<DNSToSwitchMapping>empty(), timer,
DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER);
repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
// Update cluster
Set<BookieId> addrs = new HashSet<BookieId>();
addrs.add(addr1.toBookieId());
addrs.add(addr2.toBookieId());
addrs.add(addr3.toBookieId());
addrs.add(addr4.toBookieId());
addrs.add(addr5.toBookieId());
addrs.add(addr6.toBookieId());
addrs.add(addr7.toBookieId());
repp.onClusterChanged(addrs, new HashSet<BookieId>());

int ensembleSize = 3;
int writeQuorumSize = 3;
int ackQuorumSize = 2;

Set<BookieId> excludeBookies = new HashSet<>();

for (int i = 0; i < 50000; ++i) {
EnsemblePlacementPolicy.PlacementResult<List<BookieId>> ensembleResponse =
repp.newEnsemble(ensembleSize, writeQuorumSize,
ackQuorumSize, null, excludeBookies);
List<BookieId> ensemble = ensembleResponse.getResult();
if (!ensemble.contains(addr1.toBookieId())) {
fail("Failed to select bookie located on the same rack with bookie client");
}
if (ensemble.contains(addr2.toBookieId()) && ensemble.contains(addr3.toBookieId())) {
fail("addr2 and addr3 is same rack.");
}
}

//addr4 shutdown.
addrs.remove(addr5.toBookieId());
repp.onClusterChanged(addrs, new HashSet<BookieId>());
for (int i = 0; i < 50000; ++i) {
EnsemblePlacementPolicy.PlacementResult<List<BookieId>> ensembleResponse =
repp.newEnsemble(ensembleSize, writeQuorumSize,
ackQuorumSize, null, excludeBookies);
List<BookieId> ensemble = ensembleResponse.getResult();
if (!ensemble.contains(addr1.toBookieId())) {
fail("Failed to select bookie located on the same rack with bookie client");
}
}

}

@Test
public void testMinNumRacksPerWriteQuorumOfRacks() throws Exception {
int numOfRacksToCreate = 6;
Expand Down
Loading