|
@@ -5,9 +5,9 @@
|
|
* licenses this file to you under the Apache License, Version 2.0 (the
|
|
* 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.
|
|
* "License"); you may not use this file except in compliance with the License.
|
|
* You may obtain a copy of the License at
|
|
* You may obtain a copy of the License at
|
|
- * <p/>
|
|
|
|
|
|
+ * <p>
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
- * <p/>
|
|
|
|
|
|
+ * <p>
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
|
@@ -19,8 +19,6 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
|
|
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
|
@@ -38,6 +36,8 @@ import java.io.IOException;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Ozone main class sets up the network server and initializes the container
|
|
* Ozone main class sets up the network server and initializes the container
|
|
* layer.
|
|
* layer.
|
|
@@ -57,12 +57,11 @@ public class OzoneContainer {
|
|
* Creates a network endpoint and enables Ozone container.
|
|
* Creates a network endpoint and enables Ozone container.
|
|
*
|
|
*
|
|
* @param ozoneConfig - Config
|
|
* @param ozoneConfig - Config
|
|
- * @param dataSet - FsDataset.
|
|
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
public OzoneContainer(
|
|
public OzoneContainer(
|
|
- Configuration ozoneConfig,
|
|
|
|
- FsDatasetSpi<? extends FsVolumeSpi> dataSet) throws Exception {
|
|
|
|
|
|
+ Configuration ozoneConfig) throws IOException {
|
|
|
|
+ this.ozoneConfig = ozoneConfig;
|
|
List<StorageLocation> locations = new LinkedList<>();
|
|
List<StorageLocation> locations = new LinkedList<>();
|
|
String[] paths = ozoneConfig.getStrings(
|
|
String[] paths = ozoneConfig.getStrings(
|
|
OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
|
|
OzoneConfigKeys.OZONE_CONTAINER_METADATA_DIRS);
|
|
@@ -71,11 +70,9 @@ public class OzoneContainer {
|
|
locations.add(StorageLocation.parse(p));
|
|
locations.add(StorageLocation.parse(p));
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- getDataDir(dataSet, locations);
|
|
|
|
|
|
+ getDataDir(locations);
|
|
}
|
|
}
|
|
|
|
|
|
- this.ozoneConfig = ozoneConfig;
|
|
|
|
-
|
|
|
|
manager = new ContainerManagerImpl();
|
|
manager = new ContainerManagerImpl();
|
|
manager.init(this.ozoneConfig, locations);
|
|
manager.init(this.ozoneConfig, locations);
|
|
this.chunkManager = new ChunkManagerImpl(manager);
|
|
this.chunkManager = new ChunkManagerImpl(manager);
|
|
@@ -90,43 +87,44 @@ public class OzoneContainer {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Starts serving requests to ozone container.
|
|
* Starts serving requests to ozone container.
|
|
- * @throws Exception
|
|
|
|
|
|
+ *
|
|
|
|
+ * @throws IOException
|
|
*/
|
|
*/
|
|
- public void start() throws Exception {
|
|
|
|
|
|
+ public void start() throws IOException {
|
|
server.start();
|
|
server.start();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Stops the ozone container.
|
|
* Stops the ozone container.
|
|
- *
|
|
|
|
- * Shutdown logic is not very obvious from the following code.
|
|
|
|
- * if you need to modify the logic, please keep these comments in mind.
|
|
|
|
- * Here is the shutdown sequence.
|
|
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
|
|
+ * Shutdown logic is not very obvious from the following code. if you need to
|
|
|
|
+ * modify the logic, please keep these comments in mind. Here is the shutdown
|
|
|
|
+ * sequence.
|
|
|
|
+ * <p>
|
|
* 1. We shutdown the network ports.
|
|
* 1. We shutdown the network ports.
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
* 2. Now we need to wait for all requests in-flight to finish.
|
|
* 2. Now we need to wait for all requests in-flight to finish.
|
|
- *
|
|
|
|
- * 3. The container manager lock is a read-write lock with "Fairness" enabled.
|
|
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
|
|
+ * 3. The container manager lock is a read-write lock with "Fairness"
|
|
|
|
+ * enabled.
|
|
|
|
+ * <p>
|
|
* 4. This means that the waiting threads are served in a "first-come-first
|
|
* 4. This means that the waiting threads are served in a "first-come-first
|
|
* -served" manner. Please note that this applies to waiting threads only.
|
|
* -served" manner. Please note that this applies to waiting threads only.
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
* 5. Since write locks are exclusive, if we are waiting to get a lock it
|
|
* 5. Since write locks are exclusive, if we are waiting to get a lock it
|
|
* implies that we are waiting for in-flight operations to complete.
|
|
* implies that we are waiting for in-flight operations to complete.
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
* 6. if there are other write operations waiting on the reader-writer lock,
|
|
* 6. if there are other write operations waiting on the reader-writer lock,
|
|
* fairness guarantees that they will proceed before the shutdown lock
|
|
* fairness guarantees that they will proceed before the shutdown lock
|
|
* request.
|
|
* request.
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
* 7. Since all operations either take a reader or writer lock of container
|
|
* 7. Since all operations either take a reader or writer lock of container
|
|
* manager, we are guaranteed that we are the last operation since we have
|
|
* manager, we are guaranteed that we are the last operation since we have
|
|
* closed the network port, and we wait until close is successful.
|
|
* closed the network port, and we wait until close is successful.
|
|
- *
|
|
|
|
|
|
+ * <p>
|
|
* 8. We take the writer lock and call shutdown on each of the managers in
|
|
* 8. We take the writer lock and call shutdown on each of the managers in
|
|
* reverse order. That is chunkManager, keyManager and containerManager is
|
|
* reverse order. That is chunkManager, keyManager and containerManager is
|
|
* shutdown.
|
|
* shutdown.
|
|
- *
|
|
|
|
*/
|
|
*/
|
|
public void stop() {
|
|
public void stop() {
|
|
LOG.info("Attempting to stop container services.");
|
|
LOG.info("Attempting to stop container services.");
|
|
@@ -144,26 +142,14 @@ public class OzoneContainer {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Returns a paths to data dirs.
|
|
* Returns a paths to data dirs.
|
|
- * @param dataset - FSDataset.
|
|
|
|
|
|
+ *
|
|
* @param pathList - List of paths.
|
|
* @param pathList - List of paths.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- private void getDataDir(
|
|
|
|
- FsDatasetSpi<? extends FsVolumeSpi> dataset,
|
|
|
|
- List<StorageLocation> pathList) throws IOException {
|
|
|
|
- FsDatasetSpi.FsVolumeReferences references;
|
|
|
|
- try {
|
|
|
|
- synchronized (dataset) {
|
|
|
|
- references = dataset.getFsVolumeReferences();
|
|
|
|
- for (int ndx = 0; ndx < references.size(); ndx++) {
|
|
|
|
- FsVolumeSpi vol = references.get(ndx);
|
|
|
|
- pathList.add(StorageLocation.parse(vol.getBaseURI().getPath()));
|
|
|
|
- }
|
|
|
|
- references.close();
|
|
|
|
- }
|
|
|
|
- } catch (IOException ex) {
|
|
|
|
- LOG.error("Unable to get volume paths.", ex);
|
|
|
|
- throw new IOException("Internal error", ex);
|
|
|
|
|
|
+ private void getDataDir(List<StorageLocation> pathList) throws IOException {
|
|
|
|
+ for (String dir : ozoneConfig.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
|
|
|
|
+ StorageLocation location = StorageLocation.parse(dir);
|
|
|
|
+ pathList.add(location);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|