HBASE-24158 [Flakey Tests] TestAsyncTableGetMultiThreaded

Addendum to address NPE
This commit is contained in:
stack 2020-04-16 08:02:33 -07:00
parent d212dc4df0
commit 4dd7259faa
1 changed files with 8 additions and 3 deletions

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
import java.util.Arrays;
import java.util.function.Consumer;
import java.util.function.Function;
@ -45,7 +44,13 @@ final class AsyncRegionLocatorHelper {
static boolean canUpdateOnError(HRegionLocation loc, HRegionLocation oldLoc) {
// Do not need to update if no such location, or the location is newer, or the location is not
// the same with us
return oldLoc != null && oldLoc.getSeqNum() <= loc.getSeqNum() &&
if (loc == null || loc.getServerName() == null) {
return false;
}
if (oldLoc == null || oldLoc.getServerName() == null) {
return false;
}
return oldLoc.getSeqNum() <= loc.getSeqNum() &&
oldLoc.getServerName().equals(loc.getServerName());
}