HDFS-12663. Ozone: OzoneClient: Remove protobuf classes exposed to clients through OzoneBucket. Contributed by Nandakumar.
This commit is contained in:
parent
80357c0eee
commit
6291ca1e32
|
@ -20,6 +20,8 @@ package org.apache.hadoop.ozone;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.ozone.client.ReplicationFactor;
|
||||
import org.apache.hadoop.ozone.client.ReplicationType;
|
||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.ozone.client.rest.RestClient;
|
||||
import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
||||
|
@ -121,6 +123,14 @@ public final class OzoneConfigKeys {
|
|||
"ozone.client.connection.timeout.ms";
|
||||
public static final int OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT = 5000;
|
||||
|
||||
public static final String OZONE_REPLICATION = "ozone.replication";
|
||||
public static final int OZONE_REPLICATION_DEFAULT =
|
||||
ReplicationFactor.THREE.getValue();
|
||||
|
||||
public static final String OZONE_REPLICATION_TYPE = "ozone.replication.type";
|
||||
public static final String OZONE_REPLICATION_TYPE_DEFAULT =
|
||||
ReplicationType.RATIS.toString();
|
||||
|
||||
/**
|
||||
* Configuration property to configure the cache size of client list calls.
|
||||
*/
|
||||
|
|
|
@ -36,7 +36,7 @@ public final class BucketArgs {
|
|||
/**
|
||||
* Bucket Version flag.
|
||||
*/
|
||||
private Boolean isVersionEnabled;
|
||||
private Boolean versioning;
|
||||
/**
|
||||
* Type of storage to be used for this bucket.
|
||||
* [RAM_DISK, SSD, DISK, ARCHIVE]
|
||||
|
@ -45,14 +45,14 @@ public final class BucketArgs {
|
|||
|
||||
/**
|
||||
* Private constructor, constructed via builder.
|
||||
* @param isVersionEnabled Bucket version flag.
|
||||
* @param versioning Bucket version flag.
|
||||
* @param storageType Storage type to be used.
|
||||
* @param acls list of ACLs.
|
||||
*/
|
||||
private BucketArgs(Boolean isVersionEnabled, StorageType storageType,
|
||||
private BucketArgs(Boolean versioning, StorageType storageType,
|
||||
List<OzoneAcl> acls) {
|
||||
this.acls = acls;
|
||||
this.isVersionEnabled = isVersionEnabled;
|
||||
this.versioning = versioning;
|
||||
this.storageType = storageType;
|
||||
}
|
||||
|
||||
|
@ -60,8 +60,8 @@ public final class BucketArgs {
|
|||
* Returns true if bucket version is enabled, else false.
|
||||
* @return isVersionEnabled
|
||||
*/
|
||||
public Boolean isVersionEnabled() {
|
||||
return isVersionEnabled;
|
||||
public Boolean getVersioning() {
|
||||
return versioning;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -93,12 +93,12 @@ public final class BucketArgs {
|
|||
* Builder for KsmBucketInfo.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Boolean isVersionEnabled;
|
||||
private Boolean versioning;
|
||||
private StorageType storageType;
|
||||
private List<OzoneAcl> acls;
|
||||
|
||||
public BucketArgs.Builder setIsVersionEnabled(Boolean versionFlag) {
|
||||
this.isVersionEnabled = versionFlag;
|
||||
public BucketArgs.Builder setVersioning(Boolean versionFlag) {
|
||||
this.versioning = versionFlag;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -117,7 +117,7 @@ public final class BucketArgs {
|
|||
* @return instance of BucketArgs.
|
||||
*/
|
||||
public BucketArgs build() {
|
||||
return new BucketArgs(isVersionEnabled, storageType, acls);
|
||||
return new BucketArgs(versioning, storageType, acls);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,11 +22,11 @@ package org.apache.hadoop.ozone.client;
|
|||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.ozone.OzoneAcl;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
@ -51,6 +51,15 @@ public class OzoneBucket {
|
|||
* Name of the bucket.
|
||||
*/
|
||||
private final String name;
|
||||
/**
|
||||
* Default replication factor to be used while creating keys.
|
||||
*/
|
||||
private final ReplicationFactor defaultReplication;
|
||||
|
||||
/**
|
||||
* Default replication type to be used while creating keys.
|
||||
*/
|
||||
private final ReplicationType defaultReplicationType;
|
||||
/**
|
||||
* Bucket ACLs.
|
||||
*/
|
||||
|
@ -100,6 +109,12 @@ public class OzoneBucket {
|
|||
this.versioning = versioning;
|
||||
this.listCacheSize = OzoneClientUtils.getListCacheSize(conf);
|
||||
this.creationTime = creationTime;
|
||||
this.defaultReplication = ReplicationFactor.valueOf(conf.getInt(
|
||||
OzoneConfigKeys.OZONE_REPLICATION,
|
||||
OzoneConfigKeys.OZONE_REPLICATION_DEFAULT));
|
||||
this.defaultReplicationType = ReplicationType.valueOf(conf.get(
|
||||
OzoneConfigKeys.OZONE_REPLICATION_TYPE,
|
||||
OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -206,17 +221,37 @@ public class OzoneBucket {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new key in the bucket.
|
||||
* Creates a new key in the bucket, with default replication type RATIS and
|
||||
* with replication factor THREE.
|
||||
* @param key Name of the key to be created.
|
||||
* @param size Size of the data the key will point to.
|
||||
* @return OzoneOutputStream to which the data has to be written.
|
||||
* @throws IOException
|
||||
*/
|
||||
public OzoneOutputStream createKey(String key, long size, OzoneProtos
|
||||
.ReplicationType type, OzoneProtos.ReplicationFactor factor)
|
||||
public OzoneOutputStream createKey(String key, long size)
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
||||
Preconditions.checkNotNull(key);
|
||||
return createKey(key, size, defaultReplicationType, defaultReplication);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new key in the bucket.
|
||||
* @param key Name of the key to be created.
|
||||
* @param size Size of the data the key will point to.
|
||||
* @param type Replication type to be used.
|
||||
* @param factor Replication factor of the key.
|
||||
* @return OzoneOutputStream to which the data has to be written.
|
||||
* @throws IOException
|
||||
*/
|
||||
public OzoneOutputStream createKey(String key, long size,
|
||||
ReplicationType type,
|
||||
ReplicationFactor factor)
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
||||
Preconditions.checkNotNull(key);
|
||||
Preconditions.checkNotNull(type);
|
||||
Preconditions.checkNotNull(factor);
|
||||
return proxy.createKey(volumeName, name, key, size, type, factor);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,63 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.ozone.client;
|
||||
|
||||
/**
|
||||
* The replication factor to be used while writing key into ozone.
|
||||
*/
|
||||
public enum ReplicationFactor {
|
||||
ONE(1),
|
||||
THREE(3);
|
||||
|
||||
/**
|
||||
* Integer representation of replication.
|
||||
*/
|
||||
private int value;
|
||||
|
||||
/**
|
||||
* Initializes ReplicationFactor with value.
|
||||
* @param value replication value
|
||||
*/
|
||||
ReplicationFactor(int value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns enum value corresponding to the int value.
|
||||
* @param value replication value
|
||||
* @return ReplicationFactor
|
||||
*/
|
||||
public static ReplicationFactor valueOf(int value) {
|
||||
if(value == 1) {
|
||||
return ONE;
|
||||
}
|
||||
if (value == 3) {
|
||||
return THREE;
|
||||
}
|
||||
throw new IllegalArgumentException("Unsupported value: " + value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns integer representation of ReplicationFactor.
|
||||
* @return replication value
|
||||
*/
|
||||
public int getValue() {
|
||||
return value;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.ozone.client;
|
||||
|
||||
/**
|
||||
* The replication type to be used while writing key into ozone.
|
||||
*/
|
||||
public enum ReplicationType {
|
||||
RATIS,
|
||||
STAND_ALONE,
|
||||
CHAINED
|
||||
}
|
|
@ -25,10 +25,11 @@ import org.apache.hadoop.ozone.client.OzoneBucket;
|
|||
import org.apache.hadoop.ozone.client.OzoneKey;
|
||||
import org.apache.hadoop.ozone.client.OzoneQuota;
|
||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||
import org.apache.hadoop.ozone.client.ReplicationFactor;
|
||||
import org.apache.hadoop.ozone.client.ReplicationType;
|
||||
import org.apache.hadoop.ozone.client.VolumeArgs;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -258,9 +259,8 @@ public interface ClientProtocol {
|
|||
*
|
||||
*/
|
||||
OzoneOutputStream createKey(String volumeName, String bucketName,
|
||||
String keyName, long size,
|
||||
OzoneProtos.ReplicationType type,
|
||||
OzoneProtos.ReplicationFactor factor)
|
||||
String keyName, long size, ReplicationType type,
|
||||
ReplicationFactor factor)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -27,10 +27,11 @@ import org.apache.hadoop.ozone.client.OzoneBucket;
|
|||
import org.apache.hadoop.ozone.client.OzoneKey;
|
||||
import org.apache.hadoop.ozone.client.OzoneQuota;
|
||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||
import org.apache.hadoop.ozone.client.ReplicationFactor;
|
||||
import org.apache.hadoop.ozone.client.ReplicationType;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -183,9 +184,10 @@ public class RestClient implements ClientProtocol {
|
|||
* @param factor @return {@link OzoneOutputStream}
|
||||
*/
|
||||
@Override
|
||||
public OzoneOutputStream createKey(String volumeName, String bucketName,
|
||||
String keyName, long size, OzoneProtos.ReplicationType type,
|
||||
OzoneProtos.ReplicationFactor factor) throws IOException {
|
||||
public OzoneOutputStream createKey(
|
||||
String volumeName, String bucketName, String keyName, long size,
|
||||
ReplicationType type, ReplicationFactor factor)
|
||||
throws IOException {
|
||||
throw new UnsupportedOperationException("Not yet implemented.");
|
||||
}
|
||||
|
||||
|
|
|
@ -31,6 +31,8 @@ import org.apache.hadoop.ozone.client.OzoneBucket;
|
|||
import org.apache.hadoop.ozone.client.OzoneKey;
|
||||
import org.apache.hadoop.ozone.client.OzoneQuota;
|
||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||
import org.apache.hadoop.ozone.client.ReplicationFactor;
|
||||
import org.apache.hadoop.ozone.client.ReplicationType;
|
||||
import org.apache.hadoop.ozone.client.VolumeArgs;
|
||||
import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
|
||||
|
@ -297,8 +299,8 @@ public class RpcClient implements ClientProtocol {
|
|||
Preconditions.checkNotNull(bucketName);
|
||||
Preconditions.checkNotNull(bucketArgs);
|
||||
|
||||
Boolean isVersionEnabled = bucketArgs.isVersionEnabled() == null ?
|
||||
Boolean.FALSE : bucketArgs.isVersionEnabled();
|
||||
Boolean isVersionEnabled = bucketArgs.getVersioning() == null ?
|
||||
Boolean.FALSE : bucketArgs.getVersioning();
|
||||
StorageType storageType = bucketArgs.getStorageType() == null ?
|
||||
StorageType.DEFAULT : bucketArgs.getStorageType();
|
||||
List<OzoneAcl> listOfAcls = new ArrayList<>();
|
||||
|
@ -438,7 +440,7 @@ public class RpcClient implements ClientProtocol {
|
|||
@Override
|
||||
public OzoneOutputStream createKey(
|
||||
String volumeName, String bucketName, String keyName, long size,
|
||||
OzoneProtos.ReplicationType type, OzoneProtos.ReplicationFactor factor)
|
||||
ReplicationType type, ReplicationFactor factor)
|
||||
throws IOException {
|
||||
String requestId = UUID.randomUUID().toString();
|
||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
||||
|
@ -446,8 +448,8 @@ public class RpcClient implements ClientProtocol {
|
|||
.setBucketName(bucketName)
|
||||
.setKeyName(keyName)
|
||||
.setDataSize(size)
|
||||
.setType(type)
|
||||
.setFactor(factor)
|
||||
.setType(OzoneProtos.ReplicationType.valueOf(type.toString()))
|
||||
.setFactor(OzoneProtos.ReplicationFactor.valueOf(factor.getValue()))
|
||||
.build();
|
||||
|
||||
OpenKeySession openKey = keySpaceManagerClient.openKey(keyArgs);
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.hadoop.ozone.client.*;
|
|||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.util.*;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -142,8 +141,8 @@ public final class Corona extends Configured implements Tool {
|
|||
private String numOfKeys;
|
||||
private String jsonDir;
|
||||
private boolean useRatis;
|
||||
private OzoneProtos.ReplicationType type;
|
||||
private OzoneProtos.ReplicationFactor factor;
|
||||
private ReplicationType type;
|
||||
private ReplicationFactor factor;
|
||||
|
||||
private int threadPoolSize;
|
||||
private int keySize;
|
||||
|
@ -372,18 +371,18 @@ public final class Corona extends Configured implements Tool {
|
|||
|
||||
useRatis = cmdLine.hasOption(RATIS);
|
||||
|
||||
type = OzoneProtos.ReplicationType.STAND_ALONE;
|
||||
factor = OzoneProtos.ReplicationFactor.ONE;
|
||||
type = ReplicationType.STAND_ALONE;
|
||||
factor = ReplicationFactor.ONE;
|
||||
|
||||
if (useRatis) {
|
||||
type = OzoneProtos.ReplicationType.RATIS;
|
||||
type = ReplicationType.RATIS;
|
||||
int replicationFactor = Integer.parseInt(cmdLine.getOptionValue(RATIS));
|
||||
switch (replicationFactor) {
|
||||
case 1:
|
||||
factor = OzoneProtos.ReplicationFactor.ONE;
|
||||
factor = ReplicationFactor.ONE;
|
||||
break;
|
||||
case 3:
|
||||
factor = OzoneProtos.ReplicationFactor.THREE;
|
||||
factor = ReplicationFactor.THREE;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Illegal replication factor:"
|
||||
|
|
|
@ -1103,4 +1103,26 @@
|
|||
Configuration property to configure the cache size of client list calls.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ozone.replication</name>
|
||||
<value>3</value>
|
||||
<tag>OZONE, CLIENT</tag>
|
||||
<description>
|
||||
Default replication value. The actual number of replications can be
|
||||
specified when writing the key. The default is used if replication
|
||||
is not specified. Supported values: 1 and 3.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ozone.replication.type</name>
|
||||
<value>RATIS</value>
|
||||
<tag>OZONE, CLIENT</tag>
|
||||
<description>
|
||||
Default replication type to be used while writing key into ozone. The
|
||||
value can be specified when writing the key, default is used when
|
||||
nothing is specified. Supported values: RATIS, STAND_ALONE and CHAINED.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
|
@ -33,10 +33,11 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|||
import org.apache.hadoop.ozone.client.OzoneKey;
|
||||
import org.apache.hadoop.ozone.client.OzoneQuota;
|
||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||
import org.apache.hadoop.ozone.client.ReplicationFactor;
|
||||
import org.apache.hadoop.ozone.client.ReplicationType;
|
||||
import org.apache.hadoop.ozone.client.VolumeArgs;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
||||
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -198,7 +199,7 @@ public class TestOzoneRpcClient {
|
|||
store.createVolume(volumeName);
|
||||
OzoneVolume volume = store.getVolume(volumeName);
|
||||
BucketArgs.Builder builder = BucketArgs.newBuilder();
|
||||
builder.setIsVersionEnabled(true);
|
||||
builder.setVersioning(true);
|
||||
volume.createBucket(bucketName, builder.build());
|
||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||
Assert.assertEquals(bucketName, bucket.getName());
|
||||
|
@ -251,7 +252,7 @@ public class TestOzoneRpcClient {
|
|||
store.createVolume(volumeName);
|
||||
OzoneVolume volume = store.getVolume(volumeName);
|
||||
BucketArgs.Builder builder = BucketArgs.newBuilder();
|
||||
builder.setIsVersionEnabled(true)
|
||||
builder.setVersioning(true)
|
||||
.setStorageType(StorageType.SSD)
|
||||
.setAcls(acls);
|
||||
volume.createBucket(bucketName, builder.build());
|
||||
|
@ -377,8 +378,8 @@ public class TestOzoneRpcClient {
|
|||
String keyName = UUID.randomUUID().toString();
|
||||
|
||||
OzoneOutputStream out = bucket.createKey(keyName,
|
||||
value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||
ReplicationFactor.ONE);
|
||||
out.write(value.getBytes());
|
||||
out.close();
|
||||
OzoneKey key = bucket.getKey(keyName);
|
||||
|
@ -405,8 +406,8 @@ public class TestOzoneRpcClient {
|
|||
volume.createBucket(bucketName);
|
||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||
OzoneOutputStream out = bucket.createKey(keyName,
|
||||
value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||
ReplicationFactor.ONE);
|
||||
out.write(value.getBytes());
|
||||
out.close();
|
||||
OzoneKey key = bucket.getKey(keyName);
|
||||
|
@ -570,26 +571,22 @@ public class TestOzoneRpcClient {
|
|||
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
||||
OzoneOutputStream one = volAbucketA.createKey(
|
||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
one.write(value);
|
||||
one.close();
|
||||
OzoneOutputStream two = volAbucketB.createKey(
|
||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
two.write(value);
|
||||
two.close();
|
||||
OzoneOutputStream three = volBbucketA.createKey(
|
||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
three.write(value);
|
||||
three.close();
|
||||
OzoneOutputStream four = volBbucketB.createKey(
|
||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
four.write(value);
|
||||
four.close();
|
||||
}
|
||||
|
@ -603,26 +600,22 @@ public class TestOzoneRpcClient {
|
|||
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
||||
OzoneOutputStream one = volAbucketA.createKey(
|
||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
one.write(value);
|
||||
one.close();
|
||||
OzoneOutputStream two = volAbucketB.createKey(
|
||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
two.write(value);
|
||||
two.close();
|
||||
OzoneOutputStream three = volBbucketA.createKey(
|
||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
three.write(value);
|
||||
three.close();
|
||||
OzoneOutputStream four = volBbucketB.createKey(
|
||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
|
||||
OzoneProtos.ReplicationFactor.ONE);
|
||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||
four.write(value);
|
||||
four.close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue