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 */
018package org.apache.hadoop.hbase.regionserver;
019
020import java.util.Map;
021import java.util.Set;
022import java.util.concurrent.BlockingQueue;
023import java.util.concurrent.ConcurrentHashMap;
024import java.util.concurrent.Executors;
025import java.util.concurrent.LinkedBlockingQueue;
026import java.util.concurrent.ScheduledExecutorService;
027import java.util.concurrent.TimeUnit;
028import java.util.concurrent.atomic.AtomicInteger;
029import java.util.concurrent.atomic.AtomicLong;
030import java.util.concurrent.atomic.LongAdder;
031import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.hadoop.util.StringUtils;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
039
040/**
041 * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
042 * with every chunk
043 */
044@InterfaceAudience.Private
045public class ChunkCreator {
046  private static final Logger LOG = LoggerFactory.getLogger(ChunkCreator.class);
047  // monotonically increasing chunkid. Starts at 1.
048  private AtomicInteger chunkID = new AtomicInteger(1);
049  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
050  // natural ordering of the key
051  // CellChunkMap creation should convert the weak ref to hard reference
052
053  // chunk id of each chunk is the first integer written on each chunk,
054  // the header size need to be changed in case chunk id size is changed
055  public static final int SIZEOF_CHUNK_HEADER = Bytes.SIZEOF_INT;
056
057  /**
058   * Types of chunks, based on their sizes
059   */
060  public enum ChunkType {
061    // An index chunk is a small chunk, allocated from the index chunks pool.
062    // Its size is fixed and is 10% of the size of a data chunk.
063    INDEX_CHUNK,
064    // A data chunk is a regular chunk, allocated from the data chunks pool.
065    // Its size is fixed and given as input to the ChunkCreator c'tor.
066    DATA_CHUNK,
067    // A jumbo chunk isn't allocated from pool. Its size is bigger than the size of a
068    // data chunk, and is determined per chunk (meaning, there is no fixed jumbo size).
069    JUMBO_CHUNK
070  }
071
072  // mapping from chunk IDs to chunks
073  private Map<Integer, Chunk> chunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
074
075  private final boolean offheap;
076  static ChunkCreator instance;
077  static boolean chunkPoolDisabled = false;
078  private MemStoreChunkPool dataChunksPool;
079  private final int chunkSize;
080  private int indexChunkSize;
081  private MemStoreChunkPool indexChunksPool;
082
083  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
084    float initialCountPercentage, HeapMemoryManager heapMemoryManager,
085    float indexChunkSizePercentage) {
086    this.offheap = offheap;
087    this.chunkSize = chunkSize; // in case pools are not allocated
088    initializePools(chunkSize, globalMemStoreSize, poolSizePercentage, indexChunkSizePercentage,
089      initialCountPercentage, heapMemoryManager);
090  }
091
092  private void initializePools(int chunkSize, long globalMemStoreSize, float poolSizePercentage,
093    float indexChunkSizePercentage, float initialCountPercentage,
094    HeapMemoryManager heapMemoryManager) {
095    this.dataChunksPool = initializePool("data", globalMemStoreSize,
096      (1 - indexChunkSizePercentage) * poolSizePercentage, initialCountPercentage, chunkSize,
097      ChunkType.DATA_CHUNK, heapMemoryManager);
098    // The index chunks pool is needed only when the index type is CCM.
099    // Since the pools are not created at all when the index type isn't CCM,
100    // we don't need to check it here.
101    this.indexChunkSize = (int) (indexChunkSizePercentage * chunkSize);
102    this.indexChunksPool =
103      initializePool("index", globalMemStoreSize, indexChunkSizePercentage * poolSizePercentage,
104        initialCountPercentage, this.indexChunkSize, ChunkType.INDEX_CHUNK, heapMemoryManager);
105  }
106
107  /**
108   * Initializes the instance of ChunkCreator
109   * @param chunkSize              the chunkSize
110   * @param offheap                indicates if the chunk is to be created offheap or not
111   * @param globalMemStoreSize     the global memstore size
112   * @param poolSizePercentage     pool size percentage
113   * @param initialCountPercentage the initial count of the chunk pool if any
114   * @param heapMemoryManager      the heapmemory manager
115   * @return singleton MSLABChunkCreator
116   */
117  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
118      justification = "Method is called by single thread at the starting of RS")
119  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
120    float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager,
121    float indexChunkSizePercent) {
122    if (instance != null) {
123      return instance;
124    }
125    instance = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
126      initialCountPercentage, heapMemoryManager, indexChunkSizePercent);
127    return instance;
128  }
129
130  public static ChunkCreator getInstance() {
131    return instance;
132  }
133
134  /**
135   * Creates and inits a data chunk. The default implementation.
136   * @return the chunk that was initialized
137   */
138  Chunk getChunk() {
139    return getChunk(ChunkType.DATA_CHUNK);
140  }
141
142  /**
143   * Creates and inits a chunk with specific type.
144   * @return the chunk that was initialized
145   */
146  Chunk getChunk(ChunkType chunkType) {
147    switch (chunkType) {
148      case INDEX_CHUNK:
149        if (indexChunksPool == null) {
150          if (indexChunkSize <= 0) {
151            throw new IllegalArgumentException(
152              "chunkType is INDEX_CHUNK but indexChunkSize is:[" + this.indexChunkSize + "]");
153          }
154          return getChunk(chunkType, indexChunkSize);
155        } else {
156          return getChunk(chunkType, indexChunksPool.getChunkSize());
157        }
158      case DATA_CHUNK:
159        if (dataChunksPool == null) {
160          return getChunk(chunkType, chunkSize);
161        } else {
162          return getChunk(chunkType, dataChunksPool.getChunkSize());
163        }
164      default:
165        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
166    }
167  }
168
169  /**
170   * Creates and inits a chunk.
171   * @return the chunk that was initialized
172   * @param size the size of the chunk to be allocated, in bytes
173   */
174  Chunk getChunk(ChunkType chunkType, int size) {
175    Chunk chunk = null;
176    MemStoreChunkPool pool = null;
177
178    // if it is one of the pools
179    if (dataChunksPool != null && chunkType == ChunkType.DATA_CHUNK) {
180      pool = dataChunksPool;
181    } else if (indexChunksPool != null && chunkType == ChunkType.INDEX_CHUNK) {
182      pool = indexChunksPool;
183    }
184
185    // if we have a pool
186    if (pool != null) {
187      // the pool creates the chunk internally. The chunk#init() call happens here
188      chunk = pool.getChunk();
189      // the pool has run out of maxCount
190      if (chunk == null) {
191        if (LOG.isTraceEnabled()) {
192          LOG.trace("The chunk pool is full. Reached maxCount= " + pool.getMaxCount()
193            + ". Creating chunk onheap.");
194        }
195      }
196    }
197
198    if (chunk == null) {
199      chunk = createChunk(false, chunkType, size);
200    }
201
202    // now we need to actually do the expensive memory allocation step in case of a new chunk,
203    // else only the offset is set to the beginning of the chunk to accept allocations
204    chunk.init();
205    return chunk;
206  }
207
208  /**
209   * Creates and inits a chunk of a special size, bigger than a regular chunk size. Such a chunk
210   * will never come from pool and will always be on demand allocated.
211   * @return the chunk that was initialized
212   * @param jumboSize the special size to be used
213   */
214  Chunk getJumboChunk(int jumboSize) {
215    int allocSize = jumboSize + SIZEOF_CHUNK_HEADER;
216
217    if (allocSize <= this.getChunkSize(ChunkType.DATA_CHUNK)) {
218      LOG.warn("Jumbo chunk size " + jumboSize + " must be more than regular chunk size "
219        + this.getChunkSize(ChunkType.DATA_CHUNK) + ". Converting to regular chunk.");
220      return getChunk();
221    }
222    // the new chunk is going to hold the jumbo cell data and needs to be referenced by
223    // a strong map.
224    return getChunk(ChunkType.JUMBO_CHUNK, allocSize);
225  }
226
227  /**
228   * Creates the chunk either onheap or offheap
229   * @param pool      indicates if the chunks have to be created which will be used by the Pool
230   * @param chunkType whether the requested chunk is data chunk or index chunk.
231   * @param size      the size of the chunk to be allocated, in bytes
232   * @return the chunk
233   */
234  private Chunk createChunk(boolean pool, ChunkType chunkType, int size) {
235    Chunk chunk = null;
236    int id = chunkID.getAndIncrement();
237    assert id > 0;
238    // do not create offheap chunk on demand
239    if (pool && this.offheap) {
240      chunk = new OffheapChunk(size, id, chunkType, pool);
241    } else {
242      chunk = new OnheapChunk(size, id, chunkType, pool);
243    }
244
245    /**
246     * Here we always put the chunk into the {@link ChunkCreator#chunkIdMap} no matter whether the
247     * chunk is pooled or not. <br/>
248     * For {@link CompactingMemStore},because the chunk could only be acquired from
249     * {@link ChunkCreator} through {@link MemStoreLABImpl}, and
250     * {@link CompactingMemStore#indexType} could only be {@link IndexType.CHUNK_MAP} when using
251     * {@link MemStoreLABImpl}, so we must put chunk into this {@link ChunkCreator#chunkIdMap} to
252     * make sure the chunk could be got by chunkId.
253     * <p>
254     * For {@link DefaultMemStore},it is also reasonable to put the chunk in
255     * {@link ChunkCreator#chunkIdMap} because: <br/>
256     * 1.When the {@link MemStoreLAB} which created the chunk is not closed, this chunk is used by
257     * the {@link Segment} which references this {@link MemStoreLAB}, so this chunk certainly should
258     * not be GC-ed, putting the chunk in {@link ChunkCreator#chunkIdMap} does not prevent useless
259     * chunk to be GC-ed. <br/>
260     * 2.When the {@link MemStoreLAB} which created the chunk is closed, and if the chunk is not
261     * pooled, {@link ChunkCreator#removeChunk} is invoked to remove the chunk from this
262     * {@link ChunkCreator#chunkIdMap}, so there is no memory leak.
263     */
264    this.chunkIdMap.put(chunk.getId(), chunk);
265
266    return chunk;
267  }
268
269  // Chunks from pool are created covered with strong references anyway.
270  private Chunk createChunkForPool(ChunkType chunkType, int chunkSize) {
271    if (chunkSize != dataChunksPool.getChunkSize() && chunkSize != indexChunksPool.getChunkSize()) {
272      return null;
273    }
274    return createChunk(true, chunkType, chunkSize);
275  }
276
277  // Used to translate the ChunkID into a chunk ref
278  Chunk getChunk(int id) {
279    // can return null if chunk was never mapped
280    return chunkIdMap.get(id);
281  }
282
283  boolean isOffheap() {
284    return this.offheap;
285  }
286
287  private void removeChunks(Set<Integer> chunkIDs) {
288    this.chunkIdMap.keySet().removeAll(chunkIDs);
289  }
290
291  Chunk removeChunk(int chunkId) {
292    return this.chunkIdMap.remove(chunkId);
293  }
294
295  // the chunks in the chunkIdMap may already be released so we shouldn't relay
296  // on this counting for strong correctness. This method is used only in testing.
297  int numberOfMappedChunks() {
298    return this.chunkIdMap.size();
299  }
300
301  void clearChunkIds() {
302    this.chunkIdMap.clear();
303  }
304
305  /**
306   * A pool of {@link Chunk} instances. MemStoreChunkPool caches a number of retired chunks for
307   * reusing, it could decrease allocating bytes when writing, thereby optimizing the garbage
308   * collection on JVM.
309   */
310  private class MemStoreChunkPool implements HeapMemoryTuneObserver {
311    private final int chunkSize;
312    private final ChunkType chunkType;
313    private int maxCount;
314
315    // A queue of reclaimed chunks
316    private final BlockingQueue<Chunk> reclaimedChunks;
317    private final float poolSizePercentage;
318
319    /** Statistics thread schedule pool */
320    private final ScheduledExecutorService scheduleThreadPool;
321    /** Statistics thread */
322    private static final int statThreadPeriod = 60 * 5;
323    private final AtomicLong chunkCount = new AtomicLong();
324    private final LongAdder reusedChunkCount = new LongAdder();
325    private final String label;
326
327    MemStoreChunkPool(String label, int chunkSize, ChunkType chunkType, int maxCount,
328      int initialCount, float poolSizePercentage) {
329      this.label = label;
330      this.chunkSize = chunkSize;
331      this.chunkType = chunkType;
332      this.maxCount = maxCount;
333      this.poolSizePercentage = poolSizePercentage;
334      this.reclaimedChunks = new LinkedBlockingQueue<>();
335      for (int i = 0; i < initialCount; i++) {
336        Chunk chunk = createChunk(true, chunkType, chunkSize);
337        chunk.init();
338        reclaimedChunks.add(chunk);
339      }
340      chunkCount.set(initialCount);
341      final String n = Thread.currentThread().getName();
342      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
343        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
344      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
345        statThreadPeriod, TimeUnit.SECONDS);
346    }
347
348    /**
349     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we
350     * have not yet created max allowed chunks count. When we have already created max allowed
351     * chunks and no free chunks as of now, return null. It is the responsibility of the caller to
352     * make a chunk then. Note: Chunks returned by this pool must be put back to the pool after its
353     * use.
354     * @return a chunk
355     * @see #putbackChunks(Chunk)
356     */
357    Chunk getChunk() {
358      Chunk chunk = reclaimedChunks.poll();
359      if (chunk != null) {
360        chunk.reset();
361        reusedChunkCount.increment();
362      } else {
363        // Make a chunk iff we have not yet created the maxCount chunks
364        while (true) {
365          long created = this.chunkCount.get();
366          if (created < this.maxCount) {
367            if (this.chunkCount.compareAndSet(created, created + 1)) {
368              chunk = createChunkForPool(chunkType, chunkSize);
369              break;
370            }
371          } else {
372            break;
373          }
374        }
375      }
376      return chunk;
377    }
378
379    int getChunkSize() {
380      return chunkSize;
381    }
382
383    /**
384     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
385     * chunks n
386     */
387    private void putbackChunks(Chunk c) {
388      int toAdd = this.maxCount - reclaimedChunks.size();
389      if (c.isFromPool() && c.size == chunkSize && toAdd > 0) {
390        reclaimedChunks.add(c);
391      } else {
392        // remove the chunk (that is not going to pool)
393        // though it is initially from the pool or not
394        ChunkCreator.this.removeChunk(c.getId());
395      }
396    }
397
398    private class StatisticsThread extends Thread {
399      StatisticsThread() {
400        super("MemStoreChunkPool.StatisticsThread");
401        setDaemon(true);
402      }
403
404      @Override
405      public void run() {
406        logStats();
407      }
408
409      private void logStats() {
410        if (!LOG.isDebugEnabled()) return;
411        long created = chunkCount.get();
412        long reused = reusedChunkCount.sum();
413        long total = created + reused;
414        LOG.debug(
415          "{} stats (chunk size={}): current pool size={}, created chunk count={}, "
416            + "reused chunk count={}, reuseRatio={}",
417          label, chunkSize, reclaimedChunks.size(), created, reused,
418          (total == 0 ? "0" : StringUtils.formatPercent((float) reused / (float) total, 2)));
419      }
420    }
421
422    private int getMaxCount() {
423      return this.maxCount;
424    }
425
426    @Override
427    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
428      // don't do any tuning in case of offheap memstore
429      if (isOffheap()) {
430        LOG.warn("{} not tuning the chunk pool as it is offheap", label);
431        return;
432      }
433      int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
434      if (newMaxCount != this.maxCount) {
435        // We need an adjustment in the chunks numbers
436        if (newMaxCount > this.maxCount) {
437          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
438          // create and add them to Q
439          LOG.info("{} max count for chunks increased from {} to {}", this.label, this.maxCount,
440            newMaxCount);
441          this.maxCount = newMaxCount;
442        } else {
443          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
444          // itself. If the extra chunks are serving already, do not pool those when we get them
445          // back
446          LOG.info("{} max count for chunks decreased from {} to {}", this.label, this.maxCount,
447            newMaxCount);
448          this.maxCount = newMaxCount;
449          if (this.reclaimedChunks.size() > newMaxCount) {
450            synchronized (this) {
451              while (this.reclaimedChunks.size() > newMaxCount) {
452                this.reclaimedChunks.poll();
453              }
454            }
455          }
456        }
457      }
458    }
459  }
460
461  static void clearDisableFlag() {
462    chunkPoolDisabled = false;
463  }
464
465  private MemStoreChunkPool initializePool(String label, long globalMemStoreSize,
466    float poolSizePercentage, float initialCountPercentage, int chunkSize, ChunkType chunkType,
467    HeapMemoryManager heapMemoryManager) {
468    if (poolSizePercentage <= 0) {
469      LOG.info("{} poolSizePercentage is less than 0. So not using pool", label);
470      return null;
471    }
472    if (chunkPoolDisabled) {
473      return null;
474    }
475    if (poolSizePercentage > 1.0) {
476      throw new IllegalArgumentException(
477        MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
478    }
479    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
480    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
481      throw new IllegalArgumentException(
482        label + " " + MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
483    }
484    int initialCount = (int) (initialCountPercentage * maxCount);
485    LOG.info("Allocating {} MemStoreChunkPool with chunk size {}, max count {}, initial count {}",
486      label, StringUtils.byteDesc(chunkSize), maxCount, initialCount);
487    MemStoreChunkPool memStoreChunkPool = new MemStoreChunkPool(label, chunkSize, chunkType,
488      maxCount, initialCount, poolSizePercentage);
489    if (heapMemoryManager != null && memStoreChunkPool != null) {
490      // Register with Heap Memory manager
491      heapMemoryManager.registerTuneObserver(memStoreChunkPool);
492    }
493    return memStoreChunkPool;
494  }
495
496  int getMaxCount() {
497    return getMaxCount(ChunkType.DATA_CHUNK);
498  }
499
500  int getMaxCount(ChunkType chunkType) {
501    switch (chunkType) {
502      case INDEX_CHUNK:
503        if (indexChunksPool != null) {
504          return indexChunksPool.getMaxCount();
505        }
506        break;
507      case DATA_CHUNK:
508        if (dataChunksPool != null) {
509          return dataChunksPool.getMaxCount();
510        }
511        break;
512      default:
513        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
514    }
515
516    return 0;
517  }
518
519  int getPoolSize() {
520    return getPoolSize(ChunkType.DATA_CHUNK);
521  }
522
523  int getPoolSize(ChunkType chunkType) {
524    switch (chunkType) {
525      case INDEX_CHUNK:
526        if (indexChunksPool != null) {
527          return indexChunksPool.reclaimedChunks.size();
528        }
529        break;
530      case DATA_CHUNK:
531        if (dataChunksPool != null) {
532          return dataChunksPool.reclaimedChunks.size();
533        }
534        break;
535      default:
536        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
537    }
538    return 0;
539  }
540
541  boolean isChunkInPool(int chunkId) {
542    Chunk c = getChunk(chunkId);
543    if (c == null) {
544      return false;
545    }
546
547    // chunks that are from pool will return true chunk reference not null
548    if (dataChunksPool != null && dataChunksPool.reclaimedChunks.contains(c)) {
549      return true;
550    } else if (indexChunksPool != null && indexChunksPool.reclaimedChunks.contains(c)) {
551      return true;
552    }
553    return false;
554  }
555
556  /*
557   * Only used in testing
558   */
559  void clearChunksInPool() {
560    if (dataChunksPool != null) {
561      dataChunksPool.reclaimedChunks.clear();
562    }
563    if (indexChunksPool != null) {
564      indexChunksPool.reclaimedChunks.clear();
565    }
566  }
567
568  int getChunkSize() {
569    return getChunkSize(ChunkType.DATA_CHUNK);
570  }
571
572  int getChunkSize(ChunkType chunkType) {
573    switch (chunkType) {
574      case INDEX_CHUNK:
575        if (indexChunksPool != null) {
576          return indexChunksPool.getChunkSize();
577        } else {
578          return indexChunkSize;
579        }
580      case DATA_CHUNK:
581        if (dataChunksPool != null) {
582          return dataChunksPool.getChunkSize();
583        } else { // When pools are empty
584          return chunkSize;
585        }
586      default:
587        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
588    }
589  }
590
591  synchronized void putbackChunks(Set<Integer> chunks) {
592    // if there is no pool just try to clear the chunkIdMap in case there is something
593    if (dataChunksPool == null && indexChunksPool == null) {
594      this.removeChunks(chunks);
595      return;
596    }
597
598    // if there is a pool, go over all chunk IDs that came back, the chunks may be from pool or not
599    for (int chunkID : chunks) {
600      // translate chunk ID to chunk, if chunk initially wasn't in pool
601      // this translation will (most likely) return null
602      Chunk chunk = ChunkCreator.this.getChunk(chunkID);
603      if (chunk != null) {
604        if (chunk.isFromPool() && chunk.isIndexChunk()) {
605          indexChunksPool.putbackChunks(chunk);
606        } else if (chunk.isFromPool() && chunk.isDataChunk()) {
607          dataChunksPool.putbackChunks(chunk);
608        } else {
609          // chunks which are not from one of the pools
610          // should be released without going to the pools.
611          // Removing them from chunkIdMap will cause their removal by the GC.
612          this.removeChunk(chunkID);
613        }
614      }
615      // if chunk is null, it was never covered by the chunkIdMap (and so wasn't in pool also),
616      // so we have nothing to do on its release
617    }
618    return;
619  }
620
621  MemStoreChunkPool getIndexChunksPool() {
622    return this.indexChunksPool;
623  }
624
625  MemStoreChunkPool getDataChunksPool() {
626    return this.dataChunksPool;
627  }
628
629}