001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019package org.apache.hadoop.hdfs.server.namenode; 020 021import static org.apache.hadoop.util.Time.monotonicNow; 022 023import java.io.BufferedInputStream; 024import java.io.BufferedOutputStream; 025import java.io.File; 026import java.io.FileInputStream; 027import java.io.FileOutputStream; 028import java.io.IOException; 029import java.io.InputStream; 030import java.io.OutputStream; 031import java.io.RandomAccessFile; 032import java.nio.ByteBuffer; 033import java.nio.channels.FileChannel; 034import java.security.DigestOutputStream; 035import java.security.MessageDigest; 036import java.util.ArrayList; 037import java.util.Collections; 038import java.util.Comparator; 039import java.util.Map; 040import java.util.Map.Entry; 041import java.util.Set; 042 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045import org.apache.hadoop.classification.InterfaceAudience; 046import org.apache.hadoop.conf.Configuration; 047import org.apache.hadoop.hdfs.protocol.HdfsConstants; 048import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto; 049import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto; 050import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; 051import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException; 052import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; 053import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection; 054import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; 055import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection; 056import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection; 057import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection; 058import org.apache.hadoop.hdfs.server.namenode.snapshot.FSImageFormatPBSnapshot; 059import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase; 060import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress; 061import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step; 062import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType; 063import org.apache.hadoop.hdfs.util.MD5FileUtils; 064import org.apache.hadoop.io.MD5Hash; 065import org.apache.hadoop.io.compress.CompressionCodec; 066import org.apache.hadoop.io.compress.CompressorStream; 067import org.apache.hadoop.util.LimitInputStream; 068import org.apache.hadoop.util.Time; 069 070import com.google.common.collect.Lists; 071import com.google.common.collect.Maps; 072import com.google.protobuf.CodedOutputStream; 073 074/** 075 * Utility class to read / write fsimage in protobuf format. 076 */ 077@InterfaceAudience.Private 078public final class FSImageFormatProtobuf { 079 private static final Logger LOG = LoggerFactory 080 .getLogger(FSImageFormatProtobuf.class); 081 082 public static final class LoaderContext { 083 private String[] stringTable; 084 private final ArrayList<INodeReference> refList = Lists.newArrayList(); 085 086 public String[] getStringTable() { 087 return stringTable; 088 } 089 090 public ArrayList<INodeReference> getRefList() { 091 return refList; 092 } 093 } 094 095 public static final class SaverContext { 096 public static class DeduplicationMap<E> { 097 private final Map<E, Integer> map = Maps.newHashMap(); 098 private DeduplicationMap() {} 099 100 static <T> DeduplicationMap<T> newMap() { 101 return new DeduplicationMap<T>(); 102 } 103 104 int getId(E value) { 105 if (value == null) { 106 return 0; 107 } 108 Integer v = map.get(value); 109 if (v == null) { 110 int nv = map.size() + 1; 111 map.put(value, nv); 112 return nv; 113 } 114 return v; 115 } 116 117 int size() { 118 return map.size(); 119 } 120 121 Set<Entry<E, Integer>> entrySet() { 122 return map.entrySet(); 123 } 124 } 125 private final ArrayList<INodeReference> refList = Lists.newArrayList(); 126 127 private final DeduplicationMap<String> stringMap = DeduplicationMap 128 .newMap(); 129 130 public DeduplicationMap<String> getStringMap() { 131 return stringMap; 132 } 133 134 public ArrayList<INodeReference> getRefList() { 135 return refList; 136 } 137 } 138 139 public static final class Loader implements FSImageFormat.AbstractLoader { 140 static final int MINIMUM_FILE_LENGTH = 8; 141 private final Configuration conf; 142 private final FSNamesystem fsn; 143 private final LoaderContext ctx; 144 /** The MD5 sum of the loaded file */ 145 private MD5Hash imgDigest; 146 /** The transaction ID of the last edit represented by the loaded file */ 147 private long imgTxId; 148 /** 149 * Whether the image's layout version must be the same with 150 * {@link HdfsConstants#NAMENODE_LAYOUT_VERSION}. This is only set to true 151 * when we're doing (rollingUpgrade rollback). 152 */ 153 private final boolean requireSameLayoutVersion; 154 155 Loader(Configuration conf, FSNamesystem fsn, 156 boolean requireSameLayoutVersion) { 157 this.conf = conf; 158 this.fsn = fsn; 159 this.ctx = new LoaderContext(); 160 this.requireSameLayoutVersion = requireSameLayoutVersion; 161 } 162 163 @Override 164 public MD5Hash getLoadedImageMd5() { 165 return imgDigest; 166 } 167 168 @Override 169 public long getLoadedImageTxId() { 170 return imgTxId; 171 } 172 173 public LoaderContext getLoaderContext() { 174 return ctx; 175 } 176 177 void load(File file) throws IOException { 178 long start = Time.monotonicNow(); 179 imgDigest = MD5FileUtils.computeMd5ForFile(file); 180 RandomAccessFile raFile = new RandomAccessFile(file, "r"); 181 FileInputStream fin = new FileInputStream(file); 182 try { 183 loadInternal(raFile, fin); 184 long end = Time.monotonicNow(); 185 LOG.info("Loaded FSImage in {} seconds.", (end - start) / 1000); 186 } finally { 187 fin.close(); 188 raFile.close(); 189 } 190 } 191 192 private void loadInternal(RandomAccessFile raFile, FileInputStream fin) 193 throws IOException { 194 if (!FSImageUtil.checkFileFormat(raFile)) { 195 throw new IOException("Unrecognized file format"); 196 } 197 FileSummary summary = FSImageUtil.loadSummary(raFile); 198 if (requireSameLayoutVersion && summary.getLayoutVersion() != 199 HdfsConstants.NAMENODE_LAYOUT_VERSION) { 200 throw new IOException("Image version " + summary.getLayoutVersion() + 201 " is not equal to the software version " + 202 HdfsConstants.NAMENODE_LAYOUT_VERSION); 203 } 204 205 FileChannel channel = fin.getChannel(); 206 207 FSImageFormatPBINode.Loader inodeLoader = new FSImageFormatPBINode.Loader( 208 fsn, this); 209 FSImageFormatPBSnapshot.Loader snapshotLoader = new FSImageFormatPBSnapshot.Loader( 210 fsn, this); 211 212 ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary 213 .getSectionsList()); 214 Collections.sort(sections, new Comparator<FileSummary.Section>() { 215 @Override 216 public int compare(FileSummary.Section s1, FileSummary.Section s2) { 217 SectionName n1 = SectionName.fromString(s1.getName()); 218 SectionName n2 = SectionName.fromString(s2.getName()); 219 if (n1 == null) { 220 return n2 == null ? 0 : -1; 221 } else if (n2 == null) { 222 return -1; 223 } else { 224 return n1.ordinal() - n2.ordinal(); 225 } 226 } 227 }); 228 229 StartupProgress prog = NameNode.getStartupProgress(); 230 /** 231 * beginStep() and the endStep() calls do not match the boundary of the 232 * sections. This is because that the current implementation only allows 233 * a particular step to be started for once. 234 */ 235 Step currentStep = null; 236 237 for (FileSummary.Section s : sections) { 238 channel.position(s.getOffset()); 239 InputStream in = new BufferedInputStream(new LimitInputStream(fin, 240 s.getLength())); 241 242 in = FSImageUtil.wrapInputStreamForCompression(conf, 243 summary.getCodec(), in); 244 245 String n = s.getName(); 246 247 switch (SectionName.fromString(n)) { 248 case NS_INFO: 249 loadNameSystemSection(in); 250 break; 251 case STRING_TABLE: 252 loadStringTableSection(in); 253 break; 254 case INODE: { 255 currentStep = new Step(StepType.INODES); 256 prog.beginStep(Phase.LOADING_FSIMAGE, currentStep); 257 inodeLoader.loadINodeSection(in); 258 } 259 break; 260 case INODE_REFERENCE: 261 snapshotLoader.loadINodeReferenceSection(in); 262 break; 263 case INODE_DIR: 264 inodeLoader.loadINodeDirectorySection(in); 265 break; 266 case FILES_UNDERCONSTRUCTION: 267 inodeLoader.loadFilesUnderConstructionSection(in); 268 break; 269 case SNAPSHOT: 270 snapshotLoader.loadSnapshotSection(in); 271 break; 272 case SNAPSHOT_DIFF: 273 snapshotLoader.loadSnapshotDiffSection(in); 274 break; 275 case SECRET_MANAGER: { 276 prog.endStep(Phase.LOADING_FSIMAGE, currentStep); 277 Step step = new Step(StepType.DELEGATION_TOKENS); 278 prog.beginStep(Phase.LOADING_FSIMAGE, step); 279 loadSecretManagerSection(in); 280 prog.endStep(Phase.LOADING_FSIMAGE, step); 281 } 282 break; 283 case CACHE_MANAGER: { 284 Step step = new Step(StepType.CACHE_POOLS); 285 prog.beginStep(Phase.LOADING_FSIMAGE, step); 286 loadCacheManagerSection(in); 287 prog.endStep(Phase.LOADING_FSIMAGE, step); 288 } 289 break; 290 default: 291 LOG.warn("Unrecognized section {}", n); 292 break; 293 } 294 } 295 } 296 297 private void loadNameSystemSection(InputStream in) throws IOException { 298 NameSystemSection s = NameSystemSection.parseDelimitedFrom(in); 299 fsn.setGenerationStampV1(s.getGenstampV1()); 300 fsn.setGenerationStampV2(s.getGenstampV2()); 301 fsn.setGenerationStampV1Limit(s.getGenstampV1Limit()); 302 fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId()); 303 imgTxId = s.getTransactionId(); 304 if (s.hasRollingUpgradeStartTime() 305 && fsn.getFSImage().hasRollbackFSImage()) { 306 // we set the rollingUpgradeInfo only when we make sure we have the 307 // rollback image 308 fsn.setRollingUpgradeInfo(true, s.getRollingUpgradeStartTime()); 309 } 310 } 311 312 private void loadStringTableSection(InputStream in) throws IOException { 313 StringTableSection s = StringTableSection.parseDelimitedFrom(in); 314 ctx.stringTable = new String[s.getNumEntry() + 1]; 315 for (int i = 0; i < s.getNumEntry(); ++i) { 316 StringTableSection.Entry e = StringTableSection.Entry 317 .parseDelimitedFrom(in); 318 ctx.stringTable[e.getId()] = e.getStr(); 319 } 320 } 321 322 private void loadSecretManagerSection(InputStream in) throws IOException { 323 SecretManagerSection s = SecretManagerSection.parseDelimitedFrom(in); 324 int numKeys = s.getNumKeys(), numTokens = s.getNumTokens(); 325 ArrayList<SecretManagerSection.DelegationKey> keys = Lists 326 .newArrayListWithCapacity(numKeys); 327 ArrayList<SecretManagerSection.PersistToken> tokens = Lists 328 .newArrayListWithCapacity(numTokens); 329 330 for (int i = 0; i < numKeys; ++i) 331 keys.add(SecretManagerSection.DelegationKey.parseDelimitedFrom(in)); 332 333 for (int i = 0; i < numTokens; ++i) 334 tokens.add(SecretManagerSection.PersistToken.parseDelimitedFrom(in)); 335 336 fsn.loadSecretManagerState(s, keys, tokens); 337 } 338 339 private void loadCacheManagerSection(InputStream in) throws IOException { 340 CacheManagerSection s = CacheManagerSection.parseDelimitedFrom(in); 341 ArrayList<CachePoolInfoProto> pools = Lists.newArrayListWithCapacity(s 342 .getNumPools()); 343 ArrayList<CacheDirectiveInfoProto> directives = Lists 344 .newArrayListWithCapacity(s.getNumDirectives()); 345 for (int i = 0; i < s.getNumPools(); ++i) 346 pools.add(CachePoolInfoProto.parseDelimitedFrom(in)); 347 for (int i = 0; i < s.getNumDirectives(); ++i) 348 directives.add(CacheDirectiveInfoProto.parseDelimitedFrom(in)); 349 fsn.getCacheManager().loadState( 350 new CacheManager.PersistState(s, pools, directives)); 351 } 352 353 } 354 355 public static final class Saver { 356 public static final int CHECK_CANCEL_INTERVAL = 4096; 357 358 private final SaveNamespaceContext context; 359 private final SaverContext saverContext; 360 private long currentOffset = FSImageUtil.MAGIC_HEADER.length; 361 private MD5Hash savedDigest; 362 363 private FileChannel fileChannel; 364 // OutputStream for the section data 365 private OutputStream sectionOutputStream; 366 private CompressionCodec codec; 367 private OutputStream underlyingOutputStream; 368 369 Saver(SaveNamespaceContext context) { 370 this.context = context; 371 this.saverContext = new SaverContext(); 372 } 373 374 public MD5Hash getSavedDigest() { 375 return savedDigest; 376 } 377 378 public SaveNamespaceContext getContext() { 379 return context; 380 } 381 382 public SaverContext getSaverContext() { 383 return saverContext; 384 } 385 386 public void commitSection(FileSummary.Builder summary, SectionName name) 387 throws IOException { 388 long oldOffset = currentOffset; 389 flushSectionOutputStream(); 390 391 if (codec != null) { 392 sectionOutputStream = codec.createOutputStream(underlyingOutputStream); 393 } else { 394 sectionOutputStream = underlyingOutputStream; 395 } 396 long length = fileChannel.position() - oldOffset; 397 summary.addSections(FileSummary.Section.newBuilder().setName(name.name) 398 .setLength(length).setOffset(currentOffset)); 399 currentOffset += length; 400 } 401 402 private void flushSectionOutputStream() throws IOException { 403 if (codec != null) { 404 ((CompressorStream) sectionOutputStream).finish(); 405 } 406 sectionOutputStream.flush(); 407 } 408 409 void save(File file, FSImageCompression compression) throws IOException { 410 FileOutputStream fout = new FileOutputStream(file); 411 fileChannel = fout.getChannel(); 412 try { 413 LOG.info("Saving image file {} using {}", file, compression); 414 long startTime = monotonicNow(); 415 saveInternal(fout, compression, file.getAbsolutePath()); 416 LOG.info("Image file {} of size {} bytes saved in {} seconds.", file, 417 file.length(), (monotonicNow() - startTime) / 1000); 418 } finally { 419 fout.close(); 420 } 421 } 422 423 private static void saveFileSummary(OutputStream out, FileSummary summary) 424 throws IOException { 425 summary.writeDelimitedTo(out); 426 int length = getOndiskTrunkSize(summary); 427 byte[] lengthBytes = new byte[4]; 428 ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length); 429 out.write(lengthBytes); 430 } 431 432 private void saveInodes(FileSummary.Builder summary) throws IOException { 433 FSImageFormatPBINode.Saver saver = new FSImageFormatPBINode.Saver(this, 434 summary); 435 436 saver.serializeINodeSection(sectionOutputStream); 437 saver.serializeINodeDirectorySection(sectionOutputStream); 438 saver.serializeFilesUCSection(sectionOutputStream); 439 } 440 441 private void saveSnapshots(FileSummary.Builder summary) throws IOException { 442 FSImageFormatPBSnapshot.Saver snapshotSaver = new FSImageFormatPBSnapshot.Saver( 443 this, summary, context, context.getSourceNamesystem()); 444 445 snapshotSaver.serializeSnapshotSection(sectionOutputStream); 446 // Skip snapshot-related sections when there is no snapshot. 447 if (context.getSourceNamesystem().getSnapshotManager() 448 .getNumSnapshots() > 0) { 449 snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream); 450 } 451 snapshotSaver.serializeINodeReferenceSection(sectionOutputStream); 452 } 453 454 private void saveInternal(FileOutputStream fout, 455 FSImageCompression compression, String filePath) throws IOException { 456 StartupProgress prog = NameNode.getStartupProgress(); 457 MessageDigest digester = MD5Hash.getDigester(); 458 459 underlyingOutputStream = new DigestOutputStream(new BufferedOutputStream( 460 fout), digester); 461 underlyingOutputStream.write(FSImageUtil.MAGIC_HEADER); 462 463 fileChannel = fout.getChannel(); 464 465 FileSummary.Builder b = FileSummary.newBuilder() 466 .setOndiskVersion(FSImageUtil.FILE_VERSION) 467 .setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION); 468 469 codec = compression.getImageCodec(); 470 if (codec != null) { 471 b.setCodec(codec.getClass().getCanonicalName()); 472 sectionOutputStream = codec.createOutputStream(underlyingOutputStream); 473 } else { 474 sectionOutputStream = underlyingOutputStream; 475 } 476 477 saveNameSystemSection(b); 478 // Check for cancellation right after serializing the name system section. 479 // Some unit tests, such as TestSaveNamespace#testCancelSaveNameSpace 480 // depends on this behavior. 481 context.checkCancelled(); 482 483 Step step = new Step(StepType.INODES, filePath); 484 prog.beginStep(Phase.SAVING_CHECKPOINT, step); 485 saveInodes(b); 486 saveSnapshots(b); 487 prog.endStep(Phase.SAVING_CHECKPOINT, step); 488 489 step = new Step(StepType.DELEGATION_TOKENS, filePath); 490 prog.beginStep(Phase.SAVING_CHECKPOINT, step); 491 saveSecretManagerSection(b); 492 prog.endStep(Phase.SAVING_CHECKPOINT, step); 493 494 step = new Step(StepType.CACHE_POOLS, filePath); 495 prog.beginStep(Phase.SAVING_CHECKPOINT, step); 496 saveCacheManagerSection(b); 497 prog.endStep(Phase.SAVING_CHECKPOINT, step); 498 499 saveStringTableSection(b); 500 501 // We use the underlyingOutputStream to write the header. Therefore flush 502 // the buffered stream (which is potentially compressed) first. 503 flushSectionOutputStream(); 504 505 FileSummary summary = b.build(); 506 saveFileSummary(underlyingOutputStream, summary); 507 underlyingOutputStream.close(); 508 savedDigest = new MD5Hash(digester.digest()); 509 } 510 511 private void saveSecretManagerSection(FileSummary.Builder summary) 512 throws IOException { 513 final FSNamesystem fsn = context.getSourceNamesystem(); 514 DelegationTokenSecretManager.SecretManagerState state = fsn 515 .saveSecretManagerState(); 516 state.section.writeDelimitedTo(sectionOutputStream); 517 for (SecretManagerSection.DelegationKey k : state.keys) 518 k.writeDelimitedTo(sectionOutputStream); 519 520 for (SecretManagerSection.PersistToken t : state.tokens) 521 t.writeDelimitedTo(sectionOutputStream); 522 523 commitSection(summary, SectionName.SECRET_MANAGER); 524 } 525 526 private void saveCacheManagerSection(FileSummary.Builder summary) 527 throws IOException { 528 final FSNamesystem fsn = context.getSourceNamesystem(); 529 CacheManager.PersistState state = fsn.getCacheManager().saveState(); 530 state.section.writeDelimitedTo(sectionOutputStream); 531 532 for (CachePoolInfoProto p : state.pools) 533 p.writeDelimitedTo(sectionOutputStream); 534 535 for (CacheDirectiveInfoProto p : state.directives) 536 p.writeDelimitedTo(sectionOutputStream); 537 538 commitSection(summary, SectionName.CACHE_MANAGER); 539 } 540 541 private void saveNameSystemSection(FileSummary.Builder summary) 542 throws IOException { 543 final FSNamesystem fsn = context.getSourceNamesystem(); 544 OutputStream out = sectionOutputStream; 545 NameSystemSection.Builder b = NameSystemSection.newBuilder() 546 .setGenstampV1(fsn.getGenerationStampV1()) 547 .setGenstampV1Limit(fsn.getGenerationStampV1Limit()) 548 .setGenstampV2(fsn.getGenerationStampV2()) 549 .setLastAllocatedBlockId(fsn.getLastAllocatedBlockId()) 550 .setTransactionId(context.getTxId()); 551 552 // We use the non-locked version of getNamespaceInfo here since 553 // the coordinating thread of saveNamespace already has read-locked 554 // the namespace for us. If we attempt to take another readlock 555 // from the actual saver thread, there's a potential of a 556 // fairness-related deadlock. See the comments on HDFS-2223. 557 b.setNamespaceId(fsn.unprotectedGetNamespaceInfo().getNamespaceID()); 558 if (fsn.isRollingUpgrade()) { 559 b.setRollingUpgradeStartTime(fsn.getRollingUpgradeInfo().getStartTime()); 560 } 561 NameSystemSection s = b.build(); 562 s.writeDelimitedTo(out); 563 564 commitSection(summary, SectionName.NS_INFO); 565 } 566 567 private void saveStringTableSection(FileSummary.Builder summary) 568 throws IOException { 569 OutputStream out = sectionOutputStream; 570 StringTableSection.Builder b = StringTableSection.newBuilder() 571 .setNumEntry(saverContext.stringMap.size()); 572 b.build().writeDelimitedTo(out); 573 for (Entry<String, Integer> e : saverContext.stringMap.entrySet()) { 574 StringTableSection.Entry.Builder eb = StringTableSection.Entry 575 .newBuilder().setId(e.getValue()).setStr(e.getKey()); 576 eb.build().writeDelimitedTo(out); 577 } 578 commitSection(summary, SectionName.STRING_TABLE); 579 } 580 } 581 582 /** 583 * Supported section name. The order of the enum determines the order of 584 * loading. 585 */ 586 public enum SectionName { 587 NS_INFO("NS_INFO"), 588 STRING_TABLE("STRING_TABLE"), 589 EXTENDED_ACL("EXTENDED_ACL"), 590 INODE("INODE"), 591 INODE_REFERENCE("INODE_REFERENCE"), 592 SNAPSHOT("SNAPSHOT"), 593 INODE_DIR("INODE_DIR"), 594 FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"), 595 SNAPSHOT_DIFF("SNAPSHOT_DIFF"), 596 SECRET_MANAGER("SECRET_MANAGER"), 597 CACHE_MANAGER("CACHE_MANAGER"); 598 599 private static final SectionName[] values = SectionName.values(); 600 601 public static SectionName fromString(String name) { 602 for (SectionName n : values) { 603 if (n.name.equals(name)) 604 return n; 605 } 606 return null; 607 } 608 609 private final String name; 610 611 private SectionName(String name) { 612 this.name = name; 613 } 614 } 615 616 private static int getOndiskTrunkSize(com.google.protobuf.GeneratedMessage s) { 617 return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize()) 618 + s.getSerializedSize(); 619 } 620 621 private FSImageFormatProtobuf() { 622 } 623}