-
Notifications
You must be signed in to change notification settings - Fork 475
/
KeyManagerImpl.java
2165 lines (1985 loc) · 84.5 KB
/
KeyManagerImpl.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.om;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.security.GeneralSecurityException;
import java.security.PrivilegedExceptionAction;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.UniqueId;
import org.apache.hadoop.hdds.utils.db.BatchOperation;
import org.apache.hadoop.hdds.utils.db.CodecRegistry;
import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.hdds.utils.db.RDBStore;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.hdds.utils.db.TableIterator;
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUpload;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteList;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
import org.apache.hadoop.ozone.security.OzoneBlockTokenSecretManager;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.ozone.security.acl.RequestContext;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.commons.codec.digest.DigestUtils;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_LIST_TRASH_KEYS_MAX;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_LIST_TRASH_KEYS_MAX_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DIRECTORY_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INTERNAL_ERROR;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KMS_PROVIDER;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.SCM_GET_PIPELINE_EXCEPTION;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
import static org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType.KEY;
import static org.apache.hadoop.util.Time.monotonicNow;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implementation of keyManager.
*/
public class KeyManagerImpl implements KeyManager {
private static final Logger LOG =
LoggerFactory.getLogger(KeyManagerImpl.class);
/**
* A SCM block client, used to talk to SCM to allocate block during putKey.
*/
private final OzoneManager ozoneManager;
private final ScmClient scmClient;
private final OMMetadataManager metadataManager;
private final long scmBlockSize;
private final boolean useRatis;
private final int preallocateBlocksMax;
private final int listTrashKeysMax;
private final String omId;
private final OzoneBlockTokenSecretManager secretManager;
private final boolean grpcBlockTokenEnabled;
private BackgroundService keyDeletingService;
private final KeyProviderCryptoExtension kmsProvider;
private final PrefixManager prefixManager;
@VisibleForTesting
public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
OMMetadataManager metadataManager, OzoneConfiguration conf, String omId,
OzoneBlockTokenSecretManager secretManager) {
this(null, new ScmClient(scmBlockClient, null), metadataManager,
conf, omId, secretManager, null, null);
}
public KeyManagerImpl(OzoneManager om, ScmClient scmClient,
OzoneConfiguration conf, String omId) {
this (om, scmClient, om.getMetadataManager(), conf, omId,
om.getBlockTokenMgr(), om.getKmsProvider(), om.getPrefixManager());
}
@SuppressWarnings("parameternumber")
public KeyManagerImpl(OzoneManager om, ScmClient scmClient,
OMMetadataManager metadataManager, OzoneConfiguration conf, String omId,
OzoneBlockTokenSecretManager secretManager,
KeyProviderCryptoExtension kmsProvider, PrefixManager prefixManager) {
this.scmBlockSize = (long) conf
.getStorageSize(OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT,
StorageUnit.BYTES);
this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
this.preallocateBlocksMax = conf.getInt(
OZONE_KEY_PREALLOCATION_BLOCKS_MAX,
OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT);
this.grpcBlockTokenEnabled = conf.getBoolean(
HDDS_BLOCK_TOKEN_ENABLED,
HDDS_BLOCK_TOKEN_ENABLED_DEFAULT);
this.listTrashKeysMax = conf.getInt(
OZONE_CLIENT_LIST_TRASH_KEYS_MAX,
OZONE_CLIENT_LIST_TRASH_KEYS_MAX_DEFAULT);
this.ozoneManager = om;
this.omId = omId;
this.scmClient = scmClient;
this.metadataManager = metadataManager;
this.prefixManager = prefixManager;
this.secretManager = secretManager;
this.kmsProvider = kmsProvider;
}
@Override
public void start(OzoneConfiguration configuration) {
if (keyDeletingService == null) {
long blockDeleteInterval = configuration.getTimeDuration(
OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT,
TimeUnit.MILLISECONDS);
long serviceTimeout = configuration.getTimeDuration(
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
keyDeletingService = new KeyDeletingService(ozoneManager,
scmClient.getBlockClient(), this, blockDeleteInterval,
serviceTimeout, configuration);
keyDeletingService.start();
}
}
KeyProviderCryptoExtension getKMSProvider() {
return kmsProvider;
}
@Override
public void stop() throws IOException {
if (keyDeletingService != null) {
keyDeletingService.shutdown();
keyDeletingService = null;
}
}
private OmBucketInfo getBucketInfo(String volumeName, String bucketName)
throws IOException {
String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
return metadataManager.getBucketTable().get(bucketKey);
}
private void validateBucket(String volumeName, String bucketName)
throws IOException {
String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
// Check if bucket exists
if (metadataManager.getBucketTable().get(bucketKey) == null) {
String volumeKey = metadataManager.getVolumeKey(volumeName);
// If the volume also does not exist, we should throw volume not found
// exception
if (metadataManager.getVolumeTable().get(volumeKey) == null) {
LOG.error("volume not found: {}", volumeName);
throw new OMException("Volume not found",
VOLUME_NOT_FOUND);
}
// if the volume exists but bucket does not exist, throw bucket not found
// exception
LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
throw new OMException("Bucket not found",
BUCKET_NOT_FOUND);
}
}
/**
* Check S3 bucket exists or not.
* @param volumeName
* @param bucketName
* @throws IOException
*/
private OmBucketInfo validateS3Bucket(String volumeName, String bucketName)
throws IOException {
String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
OmBucketInfo omBucketInfo = metadataManager.getBucketTable().
get(bucketKey);
//Check if bucket already exists
if (omBucketInfo == null) {
LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
throw new OMException("Bucket not found",
BUCKET_NOT_FOUND);
}
return omBucketInfo;
}
@Override
public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID,
ExcludeList excludeList) throws IOException {
Preconditions.checkNotNull(args);
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
validateBucket(volumeName, bucketName);
String openKey = metadataManager.getOpenKey(
volumeName, bucketName, keyName, clientID);
OmKeyInfo keyInfo = metadataManager.getOpenKeyTable().get(openKey);
if (keyInfo == null) {
LOG.error("Allocate block for a key not in open status in meta store" +
" /{}/{}/{} with ID {}", volumeName, bucketName, keyName, clientID);
throw new OMException("Open Key not found",
KEY_NOT_FOUND);
}
// current version not committed, so new blocks coming now are added to
// the same version
List<OmKeyLocationInfo> locationInfos =
allocateBlock(keyInfo, excludeList, scmBlockSize);
keyInfo.appendNewBlocks(locationInfos, true);
keyInfo.updateModifcationTime();
metadataManager.getOpenKeyTable().put(openKey, keyInfo);
return locationInfos.get(0);
}
/**
* This methods avoids multiple rpc calls to SCM by allocating multiple blocks
* in one rpc call.
* @param keyInfo - key info for key to be allocated.
* @param requestedSize requested length for allocation.
* @param excludeList exclude list while allocating blocks.
* @param requestedSize requested size to be allocated.
* @return
* @throws IOException
*/
private List<OmKeyLocationInfo> allocateBlock(OmKeyInfo keyInfo,
ExcludeList excludeList, long requestedSize) throws IOException {
int numBlocks = Math.min((int) ((requestedSize - 1) / scmBlockSize + 1),
preallocateBlocksMax);
List<OmKeyLocationInfo> locationInfos = new ArrayList<>(numBlocks);
String remoteUser = getRemoteUser().getShortUserName();
List<AllocatedBlock> allocatedBlocks;
try {
allocatedBlocks = scmClient.getBlockClient()
.allocateBlock(scmBlockSize, numBlocks, keyInfo.getType(),
keyInfo.getFactor(), omId, excludeList);
} catch (SCMException ex) {
if (ex.getResult()
.equals(SCMException.ResultCodes.SAFE_MODE_EXCEPTION)) {
throw new OMException(ex.getMessage(), ResultCodes.SCM_IN_SAFE_MODE);
}
throw ex;
}
for (AllocatedBlock allocatedBlock : allocatedBlocks) {
OmKeyLocationInfo.Builder builder = new OmKeyLocationInfo.Builder()
.setBlockID(new BlockID(allocatedBlock.getBlockID()))
.setLength(scmBlockSize)
.setOffset(0)
.setPipeline(allocatedBlock.getPipeline());
if (grpcBlockTokenEnabled) {
builder.setToken(secretManager
.generateToken(remoteUser, allocatedBlock.getBlockID().toString(),
getAclForUser(remoteUser), scmBlockSize));
}
locationInfos.add(builder.build());
}
return locationInfos;
}
/* Optimize ugi lookup for RPC operations to avoid a trip through
* UGI.getCurrentUser which is synch'ed.
*/
public static UserGroupInformation getRemoteUser() throws IOException {
UserGroupInformation ugi = Server.getRemoteUser();
return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
}
/**
* Return acl for user.
* @param user
*
* */
private EnumSet<AccessModeProto> getAclForUser(String user) {
// TODO: Return correct acl for user.
return EnumSet.allOf(AccessModeProto.class);
}
private EncryptedKeyVersion generateEDEK(
final String ezKeyName) throws IOException {
if (ezKeyName == null) {
return null;
}
long generateEDEKStartTime = monotonicNow();
EncryptedKeyVersion edek = SecurityUtil.doAsLoginUser(
new PrivilegedExceptionAction<EncryptedKeyVersion>() {
@Override
public EncryptedKeyVersion run() throws IOException {
try {
return getKMSProvider().generateEncryptedKey(ezKeyName);
} catch (GeneralSecurityException e) {
throw new IOException(e);
}
}
});
long generateEDEKTime = monotonicNow() - generateEDEKStartTime;
LOG.debug("generateEDEK takes {} ms", generateEDEKTime);
Preconditions.checkNotNull(edek);
return edek;
}
@Override
public OpenKeySession openKey(OmKeyArgs args) throws IOException {
Preconditions.checkNotNull(args);
Preconditions.checkNotNull(args.getAcls(), "Default acls " +
"should be set.");
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
validateBucket(volumeName, bucketName);
long currentTime = UniqueId.next();
OmKeyInfo keyInfo;
long openVersion;
// NOTE size of a key is not a hard limit on anything, it is a value that
// client should expect, in terms of current size of key. If client sets
// a value, then this value is used, otherwise, we allocate a single
// block which is the current size, if read by the client.
final long size = args.getDataSize() > 0 ?
args.getDataSize() : scmBlockSize;
final List<OmKeyLocationInfo> locations = new ArrayList<>();
ReplicationFactor factor = args.getFactor();
if (factor == null) {
factor = useRatis ? ReplicationFactor.THREE : ReplicationFactor.ONE;
}
ReplicationType type = args.getType();
if (type == null) {
type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
}
String dbKeyName = metadataManager.getOzoneKey(
args.getVolumeName(), args.getBucketName(), args.getKeyName());
FileEncryptionInfo encInfo;
metadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName, bucketName);
OmBucketInfo bucketInfo;
try {
bucketInfo = getBucketInfo(volumeName, bucketName);
encInfo = getFileEncryptionInfo(bucketInfo);
keyInfo = prepareKeyInfo(args, dbKeyName, size, locations, encInfo);
} catch (OMException e) {
throw e;
} catch (IOException ex) {
LOG.error("Key open failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(), ResultCodes.KEY_ALLOCATION_ERROR);
} finally {
metadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
bucketName);
}
if (keyInfo == null) {
// the key does not exist, create a new object, the new blocks are the
// version 0
keyInfo = createKeyInfo(args, locations, factor, type, size,
encInfo, bucketInfo);
}
openVersion = keyInfo.getLatestVersionLocations().getVersion();
LOG.debug("Key {} allocated in volume {} bucket {}",
keyName, volumeName, bucketName);
allocateBlockInKey(keyInfo, size, currentTime);
return new OpenKeySession(currentTime, keyInfo, openVersion);
}
private void allocateBlockInKey(OmKeyInfo keyInfo, long size, long sessionId)
throws IOException {
String openKey = metadataManager
.getOpenKey(keyInfo.getVolumeName(), keyInfo.getBucketName(),
keyInfo.getKeyName(), sessionId);
// requested size is not required but more like a optimization:
// SCM looks at the requested, if it 0, no block will be allocated at
// the point, if client needs more blocks, client can always call
// allocateBlock. But if requested size is not 0, OM will preallocate
// some blocks and piggyback to client, to save RPC calls.
if (size > 0) {
List<OmKeyLocationInfo> locationInfos =
allocateBlock(keyInfo, new ExcludeList(), size);
keyInfo.appendNewBlocks(locationInfos, true);
}
metadataManager.getOpenKeyTable().put(openKey, keyInfo);
}
private OmKeyInfo prepareKeyInfo(
OmKeyArgs keyArgs, String dbKeyName, long size,
List<OmKeyLocationInfo> locations, FileEncryptionInfo encInfo)
throws IOException {
OmKeyInfo keyInfo = null;
if (keyArgs.getIsMultipartKey()) {
keyInfo = prepareMultipartKeyInfo(keyArgs, size, locations, encInfo);
} else if (metadataManager.getKeyTable().isExist(dbKeyName)) {
keyInfo = metadataManager.getKeyTable().get(dbKeyName);
// the key already exist, the new blocks will be added as new version
// when locations.size = 0, the new version will have identical blocks
// as its previous version
keyInfo.addNewVersion(locations, true);
keyInfo.setDataSize(size + keyInfo.getDataSize());
}
if(keyInfo != null) {
keyInfo.setMetadata(keyArgs.getMetadata());
}
return keyInfo;
}
private OmKeyInfo prepareMultipartKeyInfo(OmKeyArgs args, long size,
List<OmKeyLocationInfo> locations, FileEncryptionInfo encInfo)
throws IOException {
ReplicationFactor factor;
ReplicationType type;
Preconditions.checkArgument(args.getMultipartUploadPartNumber() > 0,
"PartNumber Should be greater than zero");
// When key is multipart upload part key, we should take replication
// type and replication factor from original key which has done
// initiate multipart upload. If we have not found any such, we throw
// error no such multipart upload.
String uploadID = args.getMultipartUploadID();
Preconditions.checkNotNull(uploadID);
String multipartKey = metadataManager
.getMultipartKey(args.getVolumeName(), args.getBucketName(),
args.getKeyName(), uploadID);
OmKeyInfo partKeyInfo = metadataManager.getOpenKeyTable().get(
multipartKey);
if (partKeyInfo == null) {
throw new OMException("No such Multipart upload is with specified " +
"uploadId " + uploadID,
ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
factor = partKeyInfo.getFactor();
type = partKeyInfo.getType();
}
// For this upload part we don't need to check in KeyTable. As this
// is not an actual key, it is a part of the key.
return createKeyInfo(args, locations, factor, type, size, encInfo,
getBucketInfo(args.getVolumeName(), args.getBucketName()));
}
/**
* Create OmKeyInfo object.
* @param keyArgs
* @param locations
* @param factor
* @param type
* @param size
* @param encInfo
* @param omBucketInfo
* @return
*/
private OmKeyInfo createKeyInfo(OmKeyArgs keyArgs,
List<OmKeyLocationInfo> locations,
ReplicationFactor factor,
ReplicationType type, long size,
FileEncryptionInfo encInfo,
OmBucketInfo omBucketInfo) {
OmKeyInfo.Builder builder = new OmKeyInfo.Builder()
.setVolumeName(keyArgs.getVolumeName())
.setBucketName(keyArgs.getBucketName())
.setKeyName(keyArgs.getKeyName())
.setOmKeyLocationInfos(Collections.singletonList(
new OmKeyLocationInfoGroup(0, locations)))
.setCreationTime(Time.now())
.setModificationTime(Time.now())
.setDataSize(size)
.setReplicationType(type)
.setReplicationFactor(factor)
.setFileEncryptionInfo(encInfo)
.addAllMetadata(keyArgs.getMetadata());
builder.setAcls(getAclsForKey(keyArgs, omBucketInfo));
if(Boolean.valueOf(omBucketInfo.getMetadata().get(OzoneConsts.GDPR_FLAG))) {
builder.addMetadata(OzoneConsts.GDPR_FLAG, Boolean.TRUE.toString());
}
return builder.build();
}
@Override
public void commitKey(OmKeyArgs args, long clientID) throws IOException {
Preconditions.checkNotNull(args);
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
List<OmKeyLocationInfo> locationInfoList = args.getLocationInfoList();
String objectKey = metadataManager
.getOzoneKey(volumeName, bucketName, keyName);
String openKey = metadataManager
.getOpenKey(volumeName, bucketName, keyName, clientID);
Preconditions.checkNotNull(locationInfoList);
try {
metadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName,
bucketName);
validateBucket(volumeName, bucketName);
OmKeyInfo keyInfo = metadataManager.getOpenKeyTable().get(openKey);
if (keyInfo == null) {
throw new OMException("Failed to commit key, as " + openKey + "entry " +
"is not found in the openKey table", KEY_NOT_FOUND);
}
keyInfo.setDataSize(args.getDataSize());
keyInfo.setModificationTime(Time.now());
//update the block length for each block
keyInfo.updateLocationInfoList(locationInfoList);
metadataManager.getStore().move(
openKey,
objectKey,
keyInfo,
metadataManager.getOpenKeyTable(),
metadataManager.getKeyTable());
} catch (OMException e) {
throw e;
} catch (IOException ex) {
LOG.error("Key commit failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
ResultCodes.KEY_ALLOCATION_ERROR);
} finally {
metadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
bucketName);
}
}
@Override
public OmKeyInfo lookupKey(OmKeyArgs args, String clientAddress)
throws IOException {
Preconditions.checkNotNull(args);
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
metadataManager.getLock().acquireReadLock(BUCKET_LOCK, volumeName,
bucketName);
try {
String keyBytes = metadataManager.getOzoneKey(
volumeName, bucketName, keyName);
OmKeyInfo value = metadataManager.getKeyTable().get(keyBytes);
if (value == null) {
LOG.debug("volume:{} bucket:{} Key:{} not found",
volumeName, bucketName, keyName);
throw new OMException("Key not found",
KEY_NOT_FOUND);
}
if (grpcBlockTokenEnabled) {
String remoteUser = getRemoteUser().getShortUserName();
for (OmKeyLocationInfoGroup key : value.getKeyLocationVersions()) {
key.getLocationList().forEach(k -> {
k.setToken(secretManager.generateToken(remoteUser,
k.getBlockID().getContainerBlockID().toString(),
getAclForUser(remoteUser),
k.getLength()));
});
}
}
// Refresh container pipeline info from SCM
// based on OmKeyArgs.refreshPipeline flag
if (args.getRefreshPipeline()) {
refreshPipeline(value);
}
if (args.getSortDatanodes()) {
sortDatanodeInPipeline(value, clientAddress);
}
return value;
} catch (IOException ex) {
if (ex instanceof OMException) {
throw ex;
}
LOG.debug("Get key failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
KEY_NOT_FOUND);
} finally {
metadataManager.getLock().releaseReadLock(BUCKET_LOCK, volumeName,
bucketName);
}
}
/**
* Refresh pipeline info in OM by asking SCM.
* @param value OmKeyInfo
*/
@VisibleForTesting
protected void refreshPipeline(OmKeyInfo value) throws IOException {
Map<Long, ContainerWithPipeline> containerWithPipelineMap = new HashMap<>();
for (OmKeyLocationInfoGroup key : value.getKeyLocationVersions()) {
for (OmKeyLocationInfo k : key.getLocationList()) {
// TODO: fix Some tests that may not initialize container client
// The production should always have containerClient initialized.
if (scmClient.getContainerClient() != null) {
try {
if (!containerWithPipelineMap.containsKey(k.getContainerID())) {
ContainerWithPipeline containerWithPipeline = scmClient
.getContainerClient()
.getContainerWithPipeline(k.getContainerID());
containerWithPipelineMap.put(k.getContainerID(),
containerWithPipeline);
}
} catch (IOException ioEx) {
LOG.debug("Get containerPipeline failed for volume:{} bucket:{} " +
"key:{}", value.getVolumeName(), value.getBucketName(),
value.getKeyName(), ioEx);
throw new OMException(ioEx.getMessage(),
SCM_GET_PIPELINE_EXCEPTION);
}
ContainerWithPipeline cp =
containerWithPipelineMap.get(k.getContainerID());
if (!cp.getPipeline().equals(k.getPipeline())) {
k.setPipeline(cp.getPipeline());
}
}
}
}
}
@Override
public void renameKey(OmKeyArgs args, String toKeyName) throws IOException {
Preconditions.checkNotNull(args);
Preconditions.checkNotNull(toKeyName);
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String fromKeyName = args.getKeyName();
if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
volumeName, bucketName, fromKeyName, toKeyName);
throw new OMException("Key name is empty",
ResultCodes.INVALID_KEY_NAME);
}
metadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName, bucketName);
try {
// fromKeyName should exist
String fromKey = metadataManager.getOzoneKey(
volumeName, bucketName, fromKeyName);
OmKeyInfo fromKeyValue = metadataManager.getKeyTable().get(fromKey);
if (fromKeyValue == null) {
// TODO: Add support for renaming open key
LOG.error(
"Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. "
+ "Key: {} not found.", volumeName, bucketName, fromKeyName,
toKeyName, fromKeyName);
throw new OMException("Key not found",
KEY_NOT_FOUND);
}
// A rename is a no-op if the target and source name is same.
// TODO: Discuss if we need to throw?.
if (fromKeyName.equals(toKeyName)) {
return;
}
// toKeyName should not exist
String toKey =
metadataManager.getOzoneKey(volumeName, bucketName, toKeyName);
OmKeyInfo toKeyValue = metadataManager.getKeyTable().get(toKey);
if (toKeyValue != null) {
LOG.error(
"Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. "
+ "Key: {} already exists.", volumeName, bucketName,
fromKeyName, toKeyName, toKeyName);
throw new OMException("Key already exists",
OMException.ResultCodes.KEY_ALREADY_EXISTS);
}
fromKeyValue.setKeyName(toKeyName);
fromKeyValue.updateModifcationTime();
DBStore store = metadataManager.getStore();
try (BatchOperation batch = store.initBatchOperation()) {
metadataManager.getKeyTable().deleteWithBatch(batch, fromKey);
metadataManager.getKeyTable().putWithBatch(batch, toKey,
fromKeyValue);
store.commitBatchOperation(batch);
}
} catch (IOException ex) {
if (ex instanceof OMException) {
throw ex;
}
LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
volumeName, bucketName, fromKeyName, toKeyName, ex);
throw new OMException(ex.getMessage(),
ResultCodes.KEY_RENAME_ERROR);
} finally {
metadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
bucketName);
}
}
@Override
public void deleteKey(OmKeyArgs args) throws IOException {
Preconditions.checkNotNull(args);
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
metadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName, bucketName);
try {
String objectKey = metadataManager.getOzoneKey(
volumeName, bucketName, keyName);
OmKeyInfo keyInfo = metadataManager.getKeyTable().get(objectKey);
if (keyInfo == null) {
throw new OMException("Key not found",
KEY_NOT_FOUND);
} else {
// directly delete key with no blocks from db. This key need not be
// moved to deleted table.
if (isKeyEmpty(keyInfo)) {
metadataManager.getKeyTable().delete(objectKey);
LOG.debug("Key {} deleted from OM DB", keyName);
return;
}
}
RepeatedOmKeyInfo repeatedOmKeyInfo =
metadataManager.getDeletedTable().get(objectKey);
repeatedOmKeyInfo = OmUtils.prepareKeyForDelete(keyInfo,
repeatedOmKeyInfo);
metadataManager.getKeyTable().delete(objectKey);
metadataManager.getDeletedTable().put(objectKey, repeatedOmKeyInfo);
} catch (OMException ex) {
throw ex;
} catch (IOException ex) {
LOG.error(String.format("Delete key failed for volume:%s "
+ "bucket:%s key:%s", volumeName, bucketName, keyName), ex);
throw new OMException(ex.getMessage(), ex,
ResultCodes.KEY_DELETION_ERROR);
} finally {
metadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
bucketName);
}
}
private boolean isKeyEmpty(OmKeyInfo keyInfo) {
for (OmKeyLocationInfoGroup keyLocationList : keyInfo
.getKeyLocationVersions()) {
if (keyLocationList.getLocationList().size() != 0) {
return false;
}
}
return true;
}
@Override
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
String startKey, String keyPrefix,
int maxKeys) throws IOException {
Preconditions.checkNotNull(volumeName);
Preconditions.checkNotNull(bucketName);
// We don't take a lock in this path, since we walk the
// underlying table using an iterator. That automatically creates a
// snapshot of the data, so we don't need these locks at a higher level
// when we iterate.
return metadataManager.listKeys(volumeName, bucketName,
startKey, keyPrefix, maxKeys);
}
@Override
public List<RepeatedOmKeyInfo> listTrash(String volumeName,
String bucketName, String startKeyName, String keyPrefix,
int maxKeys) throws IOException {
Preconditions.checkNotNull(volumeName);
Preconditions.checkNotNull(bucketName);
Preconditions.checkArgument(maxKeys <= listTrashKeysMax,
"The max keys limit specified is not less than the cluster " +
"allowed maximum limit.");
return metadataManager.listTrash(volumeName, bucketName,
startKeyName, keyPrefix, maxKeys);
}
@Override
public List<BlockGroup> getPendingDeletionKeys(final int count)
throws IOException {
return metadataManager.getPendingDeletionKeys(count);
}
@Override
public List<BlockGroup> getExpiredOpenKeys() throws IOException {
return metadataManager.getExpiredOpenKeys();
}
@Override
public void deleteExpiredOpenKey(String objectKeyName) throws IOException {
Preconditions.checkNotNull(objectKeyName);
// TODO: Fix this in later patches.
}
@Override
public OMMetadataManager getMetadataManager() {
return metadataManager;
}
@Override
public BackgroundService getDeletingService() {
return keyDeletingService;
}
@Override
public OmMultipartInfo initiateMultipartUpload(OmKeyArgs omKeyArgs) throws
IOException {
Preconditions.checkNotNull(omKeyArgs);
String uploadID = UUID.randomUUID().toString() + "-" + UniqueId.next();
return createMultipartInfo(omKeyArgs, uploadID);
}
private OmMultipartInfo createMultipartInfo(OmKeyArgs keyArgs,
String multipartUploadID) throws IOException {
String volumeName = keyArgs.getVolumeName();
String bucketName = keyArgs.getBucketName();
String keyName = keyArgs.getKeyName();
metadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName, bucketName);
OmBucketInfo bucketInfo = validateS3Bucket(volumeName, bucketName);
try {
// We are adding uploadId to key, because if multiple users try to
// perform multipart upload on the same key, each will try to upload, who
// ever finally commit the key, we see that key in ozone. Suppose if we
// don't add id, and use the same key /volume/bucket/key, when multiple
// users try to upload the key, we update the parts of the key's from
// multiple users to same key, and the key output can be a mix of the
// parts from multiple users.
// So on same key if multiple time multipart upload is initiated we
// store multiple entries in the openKey Table.
// Checked AWS S3, when we try to run multipart upload, each time a
// new uploadId is returned.
String multipartKey = metadataManager.getMultipartKey(volumeName,
bucketName, keyName, multipartUploadID);
// Not checking if there is an already key for this in the keyTable, as
// during final complete multipart upload we take care of this.
long currentTime = Time.now();
Map<Integer, PartKeyInfo> partKeyInfoMap = new HashMap<>();
OmMultipartKeyInfo multipartKeyInfo = new OmMultipartKeyInfo.Builder()
.setUploadID(multipartUploadID)
.setCreationTime(currentTime)
.setReplicationType(keyArgs.getType())
.setReplicationFactor(keyArgs.getFactor())
.setPartKeyInfoList(partKeyInfoMap)
.build();
List<OmKeyLocationInfo> locations = new ArrayList<>();
OmKeyInfo omKeyInfo = new OmKeyInfo.Builder()
.setVolumeName(keyArgs.getVolumeName())
.setBucketName(keyArgs.getBucketName())
.setKeyName(keyArgs.getKeyName())
.setCreationTime(currentTime)
.setModificationTime(currentTime)
.setReplicationType(keyArgs.getType())
.setReplicationFactor(keyArgs.getFactor())
.setOmKeyLocationInfos(Collections.singletonList(
new OmKeyLocationInfoGroup(0, locations)))
.setAcls(getAclsForKey(keyArgs, bucketInfo))
.build();
DBStore store = metadataManager.getStore();
try (BatchOperation batch = store.initBatchOperation()) {
// Create an entry in open key table and multipart info table for
// this key.
metadataManager.getMultipartInfoTable().putWithBatch(batch,
multipartKey, multipartKeyInfo);
metadataManager.getOpenKeyTable().putWithBatch(batch,
multipartKey, omKeyInfo);
store.commitBatchOperation(batch);
return new OmMultipartInfo(volumeName, bucketName, keyName,
multipartUploadID);
}
} catch (IOException ex) {
LOG.error("Initiate Multipart upload Failed for volume:{} bucket:{} " +
"key:{}", volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
ResultCodes.INITIATE_MULTIPART_UPLOAD_ERROR);
} finally {
metadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
bucketName);
}
}
private List<OzoneAcl> getAclsForKey(OmKeyArgs keyArgs,
OmBucketInfo bucketInfo) {
List<OzoneAcl> acls = new ArrayList<>();
if(keyArgs.getAcls() != null) {
acls.addAll(keyArgs.getAcls());
}
// Inherit DEFAULT acls from prefix.
if(prefixManager != null) {
List<OmPrefixInfo> prefixList = prefixManager.getLongestPrefixPath(
OZONE_URI_DELIMITER +
keyArgs.getVolumeName() + OZONE_URI_DELIMITER +
keyArgs.getBucketName() + OZONE_URI_DELIMITER +
keyArgs.getKeyName());
if (!prefixList.isEmpty()) {