|
@@ -1,19 +1,18 @@
|
|
-/*
|
|
|
|
- * 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.
|
|
|
|
|
|
+/**
|
|
|
|
+ * 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
|
|
|
|
+ * <p>
|
|
|
|
+ * http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
+ * <p>
|
|
|
|
+ * 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.ozone.container.common.impl;
|
|
package org.apache.hadoop.ozone.container.common.impl;
|
|
@@ -21,8 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
|
|
import org.apache.commons.codec.binary.Hex;
|
|
import org.apache.commons.codec.binary.Hex;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
@@ -31,9 +29,9 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
-import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
|
|
import org.apache.hadoop.ozone.container.common.utils.LevelDBStore;
|
|
import org.apache.hadoop.ozone.container.common.utils.LevelDBStore;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
|
|
|
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.AfterClass;
|
|
import org.junit.AfterClass;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
@@ -64,7 +62,6 @@ import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
|
|
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
|
|
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
|
|
import static org.apache.hadoop.ozone.container.ContainerTestHelper
|
|
import static org.apache.hadoop.ozone.container.ContainerTestHelper
|
|
.setDataChecksum;
|
|
.setDataChecksum;
|
|
-import static org.junit.Assert.assertArrayEquals;
|
|
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -85,9 +82,8 @@ public class TestContainerPersistence {
|
|
private static ChunkManagerImpl chunkManager;
|
|
private static ChunkManagerImpl chunkManager;
|
|
private static KeyManagerImpl keyManager;
|
|
private static KeyManagerImpl keyManager;
|
|
private static OzoneConfiguration conf;
|
|
private static OzoneConfiguration conf;
|
|
- private static FsDatasetSpi<? extends FsVolumeSpi> fsDataSet;
|
|
|
|
private static MiniOzoneCluster cluster;
|
|
private static MiniOzoneCluster cluster;
|
|
- private static List<Path> pathLists = new LinkedList<>();
|
|
|
|
|
|
+ private static List<StorageLocation> pathLists = new LinkedList<>();
|
|
|
|
|
|
@BeforeClass
|
|
@BeforeClass
|
|
public static void init() throws Throwable {
|
|
public static void init() throws Throwable {
|
|
@@ -103,12 +99,10 @@ public class TestContainerPersistence {
|
|
if (containerDir.exists()) {
|
|
if (containerDir.exists()) {
|
|
FileUtils.deleteDirectory(new File(path));
|
|
FileUtils.deleteDirectory(new File(path));
|
|
}
|
|
}
|
|
-
|
|
|
|
Assert.assertTrue(containerDir.mkdirs());
|
|
Assert.assertTrue(containerDir.mkdirs());
|
|
|
|
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
.setHandlerType("local").build();
|
|
.setHandlerType("local").build();
|
|
- fsDataSet = cluster.getDataNodes().get(0).getFSDataset();
|
|
|
|
containerManager = new ContainerManagerImpl();
|
|
containerManager = new ContainerManagerImpl();
|
|
chunkManager = new ChunkManagerImpl(containerManager);
|
|
chunkManager = new ChunkManagerImpl(containerManager);
|
|
containerManager.setChunkManager(chunkManager);
|
|
containerManager.setChunkManager(chunkManager);
|
|
@@ -130,8 +124,8 @@ public class TestContainerPersistence {
|
|
}
|
|
}
|
|
pathLists.clear();
|
|
pathLists.clear();
|
|
containerManager.getContainerMap().clear();
|
|
containerManager.getContainerMap().clear();
|
|
- pathLists.add(Paths.get(path));
|
|
|
|
- containerManager.init(conf, pathLists, fsDataSet);
|
|
|
|
|
|
+ pathLists.add(StorageLocation.parse(path.toString()));
|
|
|
|
+ containerManager.init(conf, pathLists);
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
@@ -190,8 +184,8 @@ public class TestContainerPersistence {
|
|
containerManager.createContainer(createSingleNodePipeline(containerName),
|
|
containerManager.createContainer(createSingleNodePipeline(containerName),
|
|
data);
|
|
data);
|
|
try {
|
|
try {
|
|
- containerManager.createContainer(createSingleNodePipeline
|
|
|
|
- (containerName), data);
|
|
|
|
|
|
+ containerManager.createContainer(createSingleNodePipeline(
|
|
|
|
+ containerName), data);
|
|
fail("Expected Exception not thrown.");
|
|
fail("Expected Exception not thrown.");
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
Assert.assertNotNull(ex);
|
|
Assert.assertNotNull(ex);
|
|
@@ -207,14 +201,14 @@ public class TestContainerPersistence {
|
|
ContainerData data = new ContainerData(containerName1);
|
|
ContainerData data = new ContainerData(containerName1);
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("owner)", "bilbo");
|
|
data.addMetadata("owner)", "bilbo");
|
|
- containerManager.createContainer(createSingleNodePipeline(containerName1)
|
|
|
|
- , data);
|
|
|
|
|
|
+ containerManager.createContainer(createSingleNodePipeline(containerName1),
|
|
|
|
+ data);
|
|
|
|
|
|
data = new ContainerData(containerName2);
|
|
data = new ContainerData(containerName2);
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("owner)", "bilbo");
|
|
data.addMetadata("owner)", "bilbo");
|
|
- containerManager.createContainer(createSingleNodePipeline(containerName2)
|
|
|
|
- , data);
|
|
|
|
|
|
+ containerManager.createContainer(createSingleNodePipeline(containerName2),
|
|
|
|
+ data);
|
|
|
|
|
|
|
|
|
|
Assert.assertTrue(containerManager.getContainerMap()
|
|
Assert.assertTrue(containerManager.getContainerMap()
|
|
@@ -233,8 +227,8 @@ public class TestContainerPersistence {
|
|
data = new ContainerData(containerName1);
|
|
data = new ContainerData(containerName1);
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("owner)", "bilbo");
|
|
data.addMetadata("owner)", "bilbo");
|
|
- containerManager.createContainer(createSingleNodePipeline(containerName1)
|
|
|
|
- , data);
|
|
|
|
|
|
+ containerManager.createContainer(createSingleNodePipeline(containerName1),
|
|
|
|
+ data);
|
|
|
|
|
|
// Assert we still have both containers.
|
|
// Assert we still have both containers.
|
|
Assert.assertTrue(containerManager.getContainerMap()
|
|
Assert.assertTrue(containerManager.getContainerMap()
|
|
@@ -262,8 +256,8 @@ public class TestContainerPersistence {
|
|
ContainerData data = new ContainerData(containerName);
|
|
ContainerData data = new ContainerData(containerName);
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("VOLUME", "shire");
|
|
data.addMetadata("owner)", "bilbo");
|
|
data.addMetadata("owner)", "bilbo");
|
|
- containerManager.createContainer(createSingleNodePipeline
|
|
|
|
- (containerName), data);
|
|
|
|
|
|
+ containerManager.createContainer(createSingleNodePipeline(containerName),
|
|
|
|
+ data);
|
|
testMap.put(containerName, data);
|
|
testMap.put(containerName, data);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -289,7 +283,7 @@ public class TestContainerPersistence {
|
|
}
|
|
}
|
|
|
|
|
|
private ChunkInfo writeChunkHelper(String containerName, String keyName,
|
|
private ChunkInfo writeChunkHelper(String containerName, String keyName,
|
|
- Pipeline pipeline) throws IOException,
|
|
|
|
|
|
+ Pipeline pipeline) throws IOException,
|
|
NoSuchAlgorithmException {
|
|
NoSuchAlgorithmException {
|
|
final int datalen = 1024;
|
|
final int datalen = 1024;
|
|
pipeline.setContainerName(containerName);
|
|
pipeline.setContainerName(containerName);
|