|
@@ -20,32 +20,21 @@ import com.google.common.base.Preconditions;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
|
|
import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
-import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
|
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
|
|
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
|
|
import org.apache.hadoop.ozone.protocol.proto
|
|
import org.apache.hadoop.ozone.protocol.proto
|
|
.KeySpaceManagerProtocolProtos.VolumeList;
|
|
.KeySpaceManagerProtocolProtos.VolumeList;
|
|
import org.apache.hadoop.ozone.protocol.proto
|
|
import org.apache.hadoop.ozone.protocol.proto
|
|
.KeySpaceManagerProtocolProtos.VolumeInfo;
|
|
.KeySpaceManagerProtocolProtos.VolumeInfo;
|
|
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
|
|
|
-import org.apache.hadoop.utils.LevelDBStore;
|
|
|
|
import org.iq80.leveldb.DBException;
|
|
import org.iq80.leveldb.DBException;
|
|
-import org.iq80.leveldb.Options;
|
|
|
|
-import org.iq80.leveldb.WriteBatch;
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.io.File;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.util.AbstractMap;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
-import java.util.concurrent.locks.ReadWriteLock;
|
|
|
|
-import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
+import java.util.Map;
|
|
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME;
|
|
|
|
-import static org.apache.hadoop.ozone.ksm
|
|
|
|
- .KSMConfigKeys.OZONE_KSM_DB_CACHE_SIZE_DEFAULT;
|
|
|
|
-import static org.apache.hadoop.ozone.ksm
|
|
|
|
- .KSMConfigKeys.OZONE_KSM_DB_CACHE_SIZE_MB;
|
|
|
|
import static org.apache.hadoop.ozone.ksm
|
|
import static org.apache.hadoop.ozone.ksm
|
|
.KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME_DEFAULT;
|
|
.KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME_DEFAULT;
|
|
import static org.apache.hadoop.ozone.ksm
|
|
import static org.apache.hadoop.ozone.ksm
|
|
@@ -60,9 +49,7 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(VolumeManagerImpl.class);
|
|
LoggerFactory.getLogger(VolumeManagerImpl.class);
|
|
|
|
|
|
- private final KeySpaceManager ksm;
|
|
|
|
- private final LevelDBStore store;
|
|
|
|
- private final ReadWriteLock lock;
|
|
|
|
|
|
+ private final MetadataManager metadataManager;
|
|
private final int maxUserVolumeCount;
|
|
private final int maxUserVolumeCount;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -70,30 +57,13 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
* @param conf - Ozone configuration.
|
|
* @param conf - Ozone configuration.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public VolumeManagerImpl(KeySpaceManager ksm, OzoneConfiguration conf)
|
|
|
|
- throws IOException {
|
|
|
|
- File metaDir = OzoneUtils.getScmMetadirPath(conf);
|
|
|
|
- final int cacheSize = conf.getInt(OZONE_KSM_DB_CACHE_SIZE_MB,
|
|
|
|
- OZONE_KSM_DB_CACHE_SIZE_DEFAULT);
|
|
|
|
- Options options = new Options();
|
|
|
|
- options.cacheSize(cacheSize * OzoneConsts.MB);
|
|
|
|
- File ksmDBFile = new File(metaDir.getPath(), KSM_DB_NAME);
|
|
|
|
- this.ksm = ksm;
|
|
|
|
- this.store = new LevelDBStore(ksmDBFile, options);
|
|
|
|
- lock = new ReentrantReadWriteLock();
|
|
|
|
|
|
+ public VolumeManagerImpl(MetadataManager metadataManager,
|
|
|
|
+ OzoneConfiguration conf) throws IOException {
|
|
|
|
+ this.metadataManager = metadataManager;
|
|
this.maxUserVolumeCount = conf.getInt(OZONE_KSM_USER_MAX_VOLUME,
|
|
this.maxUserVolumeCount = conf.getInt(OZONE_KSM_USER_MAX_VOLUME,
|
|
OZONE_KSM_USER_MAX_VOLUME_DEFAULT);
|
|
OZONE_KSM_USER_MAX_VOLUME_DEFAULT);
|
|
}
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
|
- public void start() {
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public void stop() throws IOException {
|
|
|
|
- store.close();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Creates a volume.
|
|
* Creates a volume.
|
|
* @param args - KsmVolumeArgs.
|
|
* @param args - KsmVolumeArgs.
|
|
@@ -101,10 +71,11 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
@Override
|
|
@Override
|
|
public void createVolume(KsmVolumeArgs args) throws IOException {
|
|
public void createVolume(KsmVolumeArgs args) throws IOException {
|
|
Preconditions.checkNotNull(args);
|
|
Preconditions.checkNotNull(args);
|
|
- lock.writeLock().lock();
|
|
|
|
- WriteBatch batch = store.createWriteBatch();
|
|
|
|
|
|
+ metadataManager.writeLock().lock();
|
|
|
|
+ List<Map.Entry<byte[], byte[]>> batch = new LinkedList<>();
|
|
try {
|
|
try {
|
|
- byte[] volumeName = store.get(DFSUtil.string2Bytes(args.getVolume()));
|
|
|
|
|
|
+ byte[] volumeName = metadataManager.
|
|
|
|
+ get(DFSUtil.string2Bytes(args.getVolume()));
|
|
|
|
|
|
// Check of the volume already exists
|
|
// Check of the volume already exists
|
|
if(volumeName != null) {
|
|
if(volumeName != null) {
|
|
@@ -114,7 +85,8 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
|
|
|
|
// Next count the number of volumes for the user
|
|
// Next count the number of volumes for the user
|
|
String dbUserName = "$" + args.getOwnerName();
|
|
String dbUserName = "$" + args.getOwnerName();
|
|
- byte[] volumeList = store.get(DFSUtil.string2Bytes(dbUserName));
|
|
|
|
|
|
+ byte[] volumeList = metadataManager
|
|
|
|
+ .get(DFSUtil.string2Bytes(dbUserName));
|
|
List prevVolList;
|
|
List prevVolList;
|
|
if (volumeList != null) {
|
|
if (volumeList != null) {
|
|
VolumeList vlist = VolumeList.parseFrom(volumeList);
|
|
VolumeList vlist = VolumeList.parseFrom(volumeList);
|
|
@@ -128,26 +100,25 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
throw new KSMException(ResultCodes.FAILED_TOO_MANY_USER_VOLUMES);
|
|
throw new KSMException(ResultCodes.FAILED_TOO_MANY_USER_VOLUMES);
|
|
}
|
|
}
|
|
|
|
|
|
- // Commit the volume information to leveldb
|
|
|
|
|
|
+ // Commit the volume information to metadataManager
|
|
VolumeInfo volumeInfo = args.getProtobuf();
|
|
VolumeInfo volumeInfo = args.getProtobuf();
|
|
- batch.put(DFSUtil.string2Bytes(args.getVolume()),
|
|
|
|
- volumeInfo.toByteArray());
|
|
|
|
|
|
+ batch.add(new AbstractMap.SimpleEntry<>(
|
|
|
|
+ DFSUtil.string2Bytes(args.getVolume()), volumeInfo.toByteArray()));
|
|
|
|
|
|
prevVolList.add(args.getVolume());
|
|
prevVolList.add(args.getVolume());
|
|
VolumeList newVolList = VolumeList.newBuilder()
|
|
VolumeList newVolList = VolumeList.newBuilder()
|
|
.addAllVolumeNames(prevVolList).build();
|
|
.addAllVolumeNames(prevVolList).build();
|
|
- batch.put(DFSUtil.string2Bytes(dbUserName), newVolList.toByteArray());
|
|
|
|
- store.commitWriteBatch(batch);
|
|
|
|
|
|
+ batch.add(new AbstractMap.SimpleEntry<>(
|
|
|
|
+ DFSUtil.string2Bytes(dbUserName), newVolList.toByteArray()));
|
|
|
|
+ metadataManager.batchPut(batch);
|
|
LOG.info("created volume:{} user:{}",
|
|
LOG.info("created volume:{} user:{}",
|
|
args.getVolume(), args.getOwnerName());
|
|
args.getVolume(), args.getOwnerName());
|
|
} catch (IOException | DBException ex) {
|
|
} catch (IOException | DBException ex) {
|
|
- ksm.getMetrics().incNumVolumeCreateFails();
|
|
|
|
LOG.error("Volume creation failed for user:{} volname:{}",
|
|
LOG.error("Volume creation failed for user:{} volname:{}",
|
|
args.getOwnerName(), args.getVolume(), ex);
|
|
args.getOwnerName(), args.getVolume(), ex);
|
|
throw ex;
|
|
throw ex;
|
|
} finally {
|
|
} finally {
|
|
- store.closeWriteBatch(batch);
|
|
|
|
- lock.writeLock().unlock();
|
|
|
|
|
|
+ metadataManager.writeLock().unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|