Skip to content
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
103 changes: 0 additions & 103 deletions hadoop-ozone/integration-test/dev-support/findbugsExcludeFile.xml
Original file line number Diff line number Diff line change
Expand Up @@ -16,107 +16,4 @@
limitations under the License.
-->
<FindBugsFilter>
<!-- For unread timeouts -->
<Match>
<Package name="~org\.apache\.hadoop.*"/>
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
</Match>
<!-- For unused File operations -->
<Match>
<Package name="~org\.apache\.hadoop.*"/>
<Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
</Match>
<!-- For read stream check -->
<Match>
<Package name="~org\.apache\.hadoop\.ozone.*"/>
<Bug pattern="RR_NOT_CHECKED" />
</Match>

<Match>
<Class name="org.apache.hadoop.fs.ozone.TestOzoneFSWithObjectStoreCreate"/>
<Bug pattern="RR_NOT_CHECKED" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdds.scm.container.TestContainerStateManagerIntegration"/>
<Bug pattern="DLS_DEAD_LOCAL_STORE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.client.rpc.TestBCSID"/>
<Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.client.rpc.TestContainerStateMachine"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.client.rpc.TestContainerStateMachineFailures"/>
<Bug pattern="RV_RETURN_VALUE_IGNORED" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.client.rpc.TestKeyInputStream"/>
<Bug pattern="SR_NOT_CHECKED" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdds.scm.storage.TestCommitWatcher"/>
<Bug pattern="URF_UNREAD_FIELD" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.container.ozoneimpl.TestSecureOzoneContainer"/>
<Bug pattern="DLS_DEAD_LOCAL_STORE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestKeyManagerImpl"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestOMRatisSnapshots"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.TestCpuMetrics"/>
<Method name="testCpuMetrics"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestOzoneManagerHAWithStoppedNodes"/>
<Bug pattern="DLS_DEAD_LOCAL_STORE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestScmSafeMode"/>
<Bug pattern="DLS_DEAD_LOCAL_STORE" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestOmAcls"/>
<Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.om.TestOmMetrics"/>
<Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdds.scm.TestContainerSmallFile"/>
<Bug pattern="DLS_DEAD_LOCAL_STORE" />
</Match>
<Match>
<Class name="org.apache.hadoop.fs.ozone.TestOzoneFileSystemMissingParent"/>
<Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD" />
</Match>
<Match>
<Class name="org.apache.hadoop.ozone.client.rpc.TestDeleteWithInAdequateDN"/>
<Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdds.scm.TestSCMInstallSnapshot"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdds.scm.TestSCMInstallSnapshotWithHA"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
</Match>
<Match>
<Class name="org.apache.hadoop.fs.contract.AbstractContractCopyFromLocalTest"/>
</Match>
<Match>
<Class name="org.apache.hadoop.fs.contract.AbstractContractEtagTest"/>
</Match>
</FindBugsFilter>
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,7 @@ public abstract class AbstractContractCopyFromLocalTest extends
@Override
public void teardown() throws Exception {
super.teardown();
if (file != null) {
file.delete();
}
FileUtils.deleteQuietly(file);
}

@Test
Expand Down Expand Up @@ -99,7 +97,7 @@ public void testCopyFileOverwrite() throws Throwable {
public void testCopyMissingFile() throws Throwable {
describe("Copying a file that's not there must fail.");
file = createTempFile("test");
file.delete();
FileUtils.deleteQuietly(file);
// first upload to create
intercept(FileNotFoundException.class, "",
() -> copyFromLocal(file, true));
Expand Down Expand Up @@ -262,7 +260,9 @@ public void testCopyDirectoryWithDelete() throws Throwable {
Files.createTempFile(srcDir, "test1", ".txt");

Path src = new Path(srcDir.toUri());
Path dst = path(srcDir.getFileName().toString());
java.nio.file.Path fileName = srcDir.getFileName();
Assertions.assertThat(fileName).isNotNull();
Path dst = path(fileName.toString());
getFileSystem().copyFromLocalFile(true, true, src, dst);

Assertions.assertThat(srcDir)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.ratis.util.Preconditions;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
Expand Down Expand Up @@ -83,7 +84,7 @@ String etagFromStatus(FileStatus st) {
Assertions.assertThat(st)
.describedAs("FileStatus %s", st)
.isInstanceOf(EtagSource.class);
final String etag = ((EtagSource) st).getEtag();
final String etag = Preconditions.assertInstanceOf(st, EtagSource.class).getEtag();
Assertions.assertThat(etag)
.describedAs("Etag of %s", st)
.isNotBlank();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.security.MessageDigest;
import java.util.ArrayList;
Expand All @@ -41,20 +42,19 @@
import java.util.List;
import java.util.Map;
import javax.xml.bind.DatatypeConverter;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdds.utils.IOUtils;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.TestDataUtil;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.om.OmConfig;
import org.apache.hadoop.ozone.om.exceptions.OMException;
Expand Down Expand Up @@ -445,19 +445,18 @@ private void createAndAssertKey(OzoneBucket ozoneBucket, String key, int length)
private void readKey(OzoneBucket ozoneBucket, String key, int length, byte[] input)
throws Exception {

OzoneInputStream ozoneInputStream = ozoneBucket.readKey(key);
byte[] read = new byte[length];
ozoneInputStream.read(read, 0, length);
ozoneInputStream.close();
try (InputStream in = ozoneBucket.readKey(key)) {
IOUtils.readFully(in, read);
}

String inputString = new String(input, UTF_8);
assertEquals(inputString, new String(read, UTF_8));

// Read using filesystem.
FSDataInputStream fsDataInputStream = o3fs.open(new Path(key));
read = new byte[length];
fsDataInputStream.read(read, 0, length);
fsDataInputStream.close();
try (InputStream in = o3fs.open(new Path(key))) {
IOUtils.readFully(in, read);
}

assertEquals(inputString, new String(read, UTF_8));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,12 @@ public void testInstallCheckPoint() throws Exception {
DBCheckpoint checkpoint = downloadSnapshot();
StorageContainerManager scm = cluster.getStorageContainerManager();
final Path location = checkpoint.getCheckpointLocation();
final DBStore db = HAUtils.loadDB(conf, location.getParent().toFile(),
location.getFileName().toString(), SCMDBDefinition.get());
Path parent = location.getParent();
assertNotNull(parent);
Path fileName = location.getFileName();
assertNotNull(fileName);
final DBStore db = HAUtils.loadDB(conf, parent.toFile(),
fileName.toString(), SCMDBDefinition.get());
// Hack the transaction index in the checkpoint so as to ensure the
// checkpointed transaction index is higher than when it was downloaded
// from.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -240,11 +240,12 @@ public void testInstallCorruptedCheckpointFailure() throws Exception {
// Corrupt the leader checkpoint and install that on the follower. The
// operation should fail and should shutdown.
boolean delete = true;
for (File file : leaderCheckpointLocation.toFile()
.listFiles()) {
File[] files = leaderCheckpointLocation.toFile().listFiles();
assertNotNull(files);
for (File file : files) {
if (file.getName().contains(".sst")) {
if (delete) {
file.delete();
FileUtils.deleteQuietly(file);
delete = false;
} else {
delete = true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import java.time.Duration;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hdds.client.BlockID;
Expand Down Expand Up @@ -86,7 +85,6 @@ public class TestCommitWatcher {
private long blockSize;
private String volumeName;
private String bucketName;
private String keyString;
private StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient;

Expand Down Expand Up @@ -147,7 +145,6 @@ public void init() throws Exception {
//the easiest way to create an open container is creating a key
client = OzoneClientFactory.getRpcClient(conf);
objectStore = client.getObjectStore();
keyString = UUID.randomUUID().toString();
volumeName = "testblockoutputstream";
bucketName = volumeName;
objectStore.createVolume(volumeName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import okhttp3.OkHttpClient;
import okhttp3.Request;
import okhttp3.Response;
import okhttp3.ResponseBody;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.ozone.test.NonHATests;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -51,7 +52,9 @@ public void testCpuMetrics() throws IOException {
// when
Response metricsResponse = httpClient.newCall(prometheusMetricsRequest)
.execute();
String metricsResponseBodyContent = metricsResponse.body().string();
ResponseBody body = metricsResponse.body();
assertThat(body).isNotNull();
String metricsResponseBodyContent = body.string();

// then
assertThat(metricsResponseBodyContent)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT_DEFAULT;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;

import com.google.common.collect.Maps;
import java.io.IOException;
Expand All @@ -30,6 +31,7 @@
import java.util.List;
import java.util.Map;
import java.util.Scanner;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
Expand Down Expand Up @@ -134,6 +136,13 @@ public static void createKey(OzoneBucket bucket, String keyName,
}
}

public static void readFully(OzoneBucket bucket, String keyName) throws IOException {
int len = Math.toIntExact(bucket.getKey(keyName).getDataSize());
try (InputStream inputStream = bucket.readKey(keyName)) {
assertDoesNotThrow(() -> IOUtils.readFully(inputStream, len));
}
}

public static String getKey(OzoneBucket bucket, String keyName)
throws IOException {
try (InputStream stream = bucket.readKey(keyName)) {
Expand Down
Loading
Loading