View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import com.google.common.annotations.VisibleForTesting;
23  import com.google.common.util.concurrent.ThreadFactoryBuilder;
24  
25  import java.io.IOException;
26  import java.util.AbstractMap.SimpleEntry;
27  import java.util.ArrayList;
28  import java.util.Collections;
29  import java.util.HashMap;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.concurrent.ConcurrentHashMap;
33  import java.util.concurrent.ExecutorService;
34  import java.util.concurrent.Executors;
35  import java.util.concurrent.LinkedBlockingQueue;
36  import java.util.concurrent.ScheduledExecutorService;
37  import java.util.concurrent.TimeUnit;
38  import java.util.concurrent.atomic.AtomicInteger;
39  import java.util.concurrent.atomic.AtomicLong;
40  
41  import org.apache.commons.logging.Log;
42  import org.apache.commons.logging.LogFactory;
43  import org.apache.hadoop.conf.Configuration;
44  import org.apache.hadoop.hbase.HBaseConfiguration;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HRegionLocation;
48  import org.apache.hadoop.hbase.ServerName;
49  import org.apache.hadoop.hbase.TableName;
50  import org.apache.hadoop.hbase.classification.InterfaceAudience;
51  import org.apache.hadoop.hbase.classification.InterfaceStability;
52  import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
53  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
54  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55  
56  /**
57   * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
58   * Each put will be sharded into different buffer queues based on its destination region server.
59   * So each region server buffer queue will only have the puts which share the same destination.
60   * And each queue will have a flush worker thread to flush the puts request to the region server.
61   * If any queue is full, the HTableMultiplexer starts to drop the Put requests for that
62   * particular queue.
63   *
64   * Also all the puts will be retried as a configuration number before dropping.
65   * And the HTableMultiplexer can report the number of buffered requests and the number of the
66   * failed (dropped) requests in total or on per region server basis.
67   *
68   * This class is thread safe.
69   */
70  @InterfaceAudience.Public
71  @InterfaceStability.Evolving
72  public class HTableMultiplexer {
73    private static final Log LOG = LogFactory.getLog(HTableMultiplexer.class.getName());
74  
75    public static final String TABLE_MULTIPLEXER_FLUSH_PERIOD_MS =
76        "hbase.tablemultiplexer.flush.period.ms";
77    public static final String TABLE_MULTIPLEXER_INIT_THREADS = "hbase.tablemultiplexer.init.threads";
78    public static final String TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE =
79        "hbase.client.max.retries.in.queue";
80  
81    /** The map between each region server to its flush worker */
82    private final Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap =
83        new ConcurrentHashMap<>();
84  
85    private final Configuration workerConf;
86    private final ClusterConnection conn;
87    private final ExecutorService pool;
88    private final int maxAttempts;
89    private final int perRegionServerBufferQueueSize;
90    private final int maxKeyValueSize;
91    private final ScheduledExecutorService executor;
92    private final long flushPeriod;
93  
94    /**
95     * @param conf The HBaseConfiguration
96     * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
97     *          each region server before dropping the request.
98     */
99    public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
100       throws IOException {
101     this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize);
102   }
103 
104   /**
105    * @param conn The HBase connection.
106    * @param conf The HBase configuration
107    * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
108    *          each region server before dropping the request.
109    */
110   public HTableMultiplexer(Connection conn, Configuration conf,
111       int perRegionServerBufferQueueSize) {
112     this.conn = (ClusterConnection) conn;
113     this.pool = HTable.getDefaultExecutor(conf);
114     // how many times we could try in total, one more than retry number
115     this.maxAttempts = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
116         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER) + 1;
117     this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
118     this.maxKeyValueSize = HTable.getMaxKeyValueSize(conf);
119     this.flushPeriod = conf.getLong(TABLE_MULTIPLEXER_FLUSH_PERIOD_MS, 100);
120     int initThreads = conf.getInt(TABLE_MULTIPLEXER_INIT_THREADS, 10);
121     this.executor =
122         Executors.newScheduledThreadPool(initThreads,
123           new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HTableFlushWorker-%d").build());
124 
125     this.workerConf = HBaseConfiguration.create(conf);
126     // We do not do the retry because we need to reassign puts to different queues if regions are
127     // moved.
128     this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
129   }
130 
131   /**
132    * Closes the internal {@link Connection}. Does nothing if the {@link Connection} has already
133    * been closed.
134    * @throws IOException If there is an error closing the connection.
135    */
136   @SuppressWarnings("deprecation")
137   public synchronized void close() throws IOException {
138     if (!getConnection().isClosed()) {
139       getConnection().close();
140     }
141   }
142 
143   /**
144    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
145    * is already full.
146    * @param tableName
147    * @param put
148    * @return true if the request can be accepted by its corresponding buffer queue.
149    */
150   public boolean put(TableName tableName, final Put put) {
151     return put(tableName, put, this.maxAttempts);
152   }
153 
154   /**
155    * The puts request will be buffered by their corresponding buffer queue.
156    * Return the list of puts which could not be queued.
157    * @param tableName
158    * @param puts
159    * @return the list of puts which could not be queued
160    */
161   public List<Put> put(TableName tableName, final List<Put> puts) {
162     if (puts == null)
163       return null;
164 
165     List <Put> failedPuts = null;
166     boolean result;
167     for (Put put : puts) {
168       result = put(tableName, put, this.maxAttempts);
169       if (result == false) {
170 
171         // Create the failed puts list if necessary
172         if (failedPuts == null) {
173           failedPuts = new ArrayList<Put>();
174         }
175         // Add the put to the failed puts list
176         failedPuts.add(put);
177       }
178     }
179     return failedPuts;
180   }
181 
182   /**
183    * @deprecated Use {@link #put(TableName, List) } instead.
184    */
185   @Deprecated
186   public List<Put> put(byte[] tableName, final List<Put> puts) {
187     return put(TableName.valueOf(tableName), puts);
188   }
189 
190   /**
191    * The put request will be buffered by its corresponding buffer queue. And the put request will be
192    * retried before dropping the request.
193    * Return false if the queue is already full.
194    * @return true if the request can be accepted by its corresponding buffer queue.
195    */
196   public boolean put(final TableName tableName, final Put put, int maxAttempts) {
197     return _put(tableName, put, maxAttempts, false);
198   }
199 
200   /**
201    * Internal "put" which exposes a boolean flag to control whether or not the region location
202    * cache should be reloaded when trying to queue the {@link Put}.
203    * @param tableName Destination table for the Put
204    * @param put The Put to send
205    * @param maxAttempts Number of attempts to retry the {@code put}
206    * @param reloadCache Should the region location cache be reloaded
207    * @return true if the request was accepted in the queue, otherwise false
208    */
209   boolean _put(final TableName tableName, final Put put, int maxAttempts, boolean reloadCache) {
210     if (maxAttempts <= 0) {
211       return false;
212     }
213 
214     try {
215       HTable.validatePut(put, maxKeyValueSize);
216       // Allow mocking to get at the connection, but don't expose the connection to users.
217       ClusterConnection conn = (ClusterConnection) getConnection();
218       HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), reloadCache);
219       if (loc != null) {
220         // Add the put pair into its corresponding queue.
221         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
222 
223         // Generate a MultiPutStatus object and offer it into the queue
224         PutStatus s = new PutStatus(loc.getRegionInfo(), put, maxAttempts);
225 
226         return queue.offer(s);
227       }
228     } catch (IOException e) {
229       LOG.debug("Cannot process the put " + put, e);
230     }
231     return false;
232   }
233 
234   /**
235    * @deprecated Use {@link #put(TableName, Put) } instead.
236    */
237   @Deprecated
238   public boolean put(final byte[] tableName, final Put put, int retry) {
239     return put(TableName.valueOf(tableName), put, retry);
240   }
241 
242   /**
243    * @deprecated Use {@link #put(TableName, Put)} instead.
244    */
245   @Deprecated
246   public boolean put(final byte[] tableName, Put put) {
247     return put(TableName.valueOf(tableName), put);
248   }
249 
250   /**
251    * @return the current HTableMultiplexerStatus
252    */
253   public HTableMultiplexerStatus getHTableMultiplexerStatus() {
254     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
255   }
256 
257   @VisibleForTesting
258   LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
259     FlushWorker worker = serverToFlushWorkerMap.get(addr);
260     if (worker == null) {
261       synchronized (this.serverToFlushWorkerMap) {
262         worker = serverToFlushWorkerMap.get(addr);
263         if (worker == null) {
264           // Create the flush worker
265           worker = new FlushWorker(workerConf, this.conn, addr, this,
266               perRegionServerBufferQueueSize, pool, executor);
267           this.serverToFlushWorkerMap.put(addr, worker);
268           executor.scheduleAtFixedRate(worker, flushPeriod, flushPeriod, TimeUnit.MILLISECONDS);
269         }
270       }
271     }
272     return worker.getQueue();
273   }
274 
275   @VisibleForTesting
276   ClusterConnection getConnection() {
277     return this.conn;
278   }
279 
280   /**
281    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
282    * report the number of buffered requests and the number of the failed (dropped) requests
283    * in total or on per region server basis.
284    */
285   @InterfaceAudience.Public
286   @InterfaceStability.Evolving
287   public static class HTableMultiplexerStatus {
288     private long totalFailedPutCounter;
289     private long totalBufferedPutCounter;
290     private long maxLatency;
291     private long overallAverageLatency;
292     private Map<String, Long> serverToFailedCounterMap;
293     private Map<String, Long> serverToBufferedCounterMap;
294     private Map<String, Long> serverToAverageLatencyMap;
295     private Map<String, Long> serverToMaxLatencyMap;
296 
297     public HTableMultiplexerStatus(
298         Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
299       this.totalBufferedPutCounter = 0;
300       this.totalFailedPutCounter = 0;
301       this.maxLatency = 0;
302       this.overallAverageLatency = 0;
303       this.serverToBufferedCounterMap = new HashMap<String, Long>();
304       this.serverToFailedCounterMap = new HashMap<String, Long>();
305       this.serverToAverageLatencyMap = new HashMap<String, Long>();
306       this.serverToMaxLatencyMap = new HashMap<String, Long>();
307       this.initialize(serverToFlushWorkerMap);
308     }
309 
310     private void initialize(
311         Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) {
312       if (serverToFlushWorkerMap == null) {
313         return;
314       }
315 
316       long averageCalcSum = 0;
317       int averageCalcCount = 0;
318       for (Map.Entry<HRegionLocation, FlushWorker> entry : serverToFlushWorkerMap
319           .entrySet()) {
320         HRegionLocation addr = entry.getKey();
321         FlushWorker worker = entry.getValue();
322 
323         long bufferedCounter = worker.getTotalBufferedCount();
324         long failedCounter = worker.getTotalFailedCount();
325         long serverMaxLatency = worker.getMaxLatency();
326         AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter();
327         // Get sum and count pieces separately to compute overall average
328         SimpleEntry<Long, Integer> averageComponents = averageCounter
329             .getComponents();
330         long serverAvgLatency = averageCounter.getAndReset();
331 
332         this.totalBufferedPutCounter += bufferedCounter;
333         this.totalFailedPutCounter += failedCounter;
334         if (serverMaxLatency > this.maxLatency) {
335           this.maxLatency = serverMaxLatency;
336         }
337         averageCalcSum += averageComponents.getKey();
338         averageCalcCount += averageComponents.getValue();
339 
340         this.serverToBufferedCounterMap.put(addr.getHostnamePort(),
341             bufferedCounter);
342         this.serverToFailedCounterMap
343             .put(addr.getHostnamePort(),
344             failedCounter);
345         this.serverToAverageLatencyMap.put(addr.getHostnamePort(),
346             serverAvgLatency);
347         this.serverToMaxLatencyMap
348             .put(addr.getHostnamePort(),
349             serverMaxLatency);
350       }
351       this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum
352           / averageCalcCount : 0;
353     }
354 
355     public long getTotalBufferedCounter() {
356       return this.totalBufferedPutCounter;
357     }
358 
359     public long getTotalFailedCounter() {
360       return this.totalFailedPutCounter;
361     }
362 
363     public long getMaxLatency() {
364       return this.maxLatency;
365     }
366 
367     public long getOverallAverageLatency() {
368       return this.overallAverageLatency;
369     }
370 
371     public Map<String, Long> getBufferedCounterForEachRegionServer() {
372       return this.serverToBufferedCounterMap;
373     }
374 
375     public Map<String, Long> getFailedCounterForEachRegionServer() {
376       return this.serverToFailedCounterMap;
377     }
378 
379     public Map<String, Long> getMaxLatencyForEachRegionServer() {
380       return this.serverToMaxLatencyMap;
381     }
382 
383     public Map<String, Long> getAverageLatencyForEachRegionServer() {
384       return this.serverToAverageLatencyMap;
385     }
386   }
387 
388   @VisibleForTesting
389   static class PutStatus {
390     final HRegionInfo regionInfo;
391     final Put put;
392     final int maxAttempCount;
393 
394     public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
395       this.regionInfo = regionInfo;
396       this.put = put;
397       this.maxAttempCount = maxAttempCount;
398     }
399   }
400 
401   /**
402    * Helper to count the average over an interval until reset.
403    */
404   private static class AtomicAverageCounter {
405     private long sum;
406     private int count;
407 
408     public AtomicAverageCounter() {
409       this.sum = 0L;
410       this.count = 0;
411     }
412 
413     public synchronized long getAndReset() {
414       long result = this.get();
415       this.reset();
416       return result;
417     }
418 
419     public synchronized long get() {
420       if (this.count == 0) {
421         return 0;
422       }
423       return this.sum / this.count;
424     }
425 
426     public synchronized SimpleEntry<Long, Integer> getComponents() {
427       return new SimpleEntry<Long, Integer>(sum, count);
428     }
429 
430     public synchronized void reset() {
431       this.sum = 0L;
432       this.count = 0;
433     }
434 
435     public synchronized void add(long value) {
436       this.sum += value;
437       this.count++;
438     }
439   }
440 
441   @VisibleForTesting
442   static class FlushWorker implements Runnable {
443     private final HRegionLocation addr;
444     private final LinkedBlockingQueue<PutStatus> queue;
445     private final HTableMultiplexer multiplexer;
446     private final AtomicLong totalFailedPutCount = new AtomicLong(0);
447     private final AtomicInteger currentProcessingCount = new AtomicInteger(0);
448     private final AtomicAverageCounter averageLatency = new AtomicAverageCounter();
449     private final AtomicLong maxLatency = new AtomicLong(0);
450 
451     private final AsyncProcess ap;
452     private final List<PutStatus> processingList = new ArrayList<>();
453     private final ScheduledExecutorService executor;
454     private final int maxRetryInQueue;
455     private final AtomicInteger retryInQueue = new AtomicInteger(0);
456 
457     public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr,
458         HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize,
459         ExecutorService pool, ScheduledExecutorService executor) {
460       this.addr = addr;
461       this.multiplexer = htableMultiplexer;
462       this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
463       RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
464       RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
465       this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory);
466       this.executor = executor;
467       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
468     }
469 
470     protected LinkedBlockingQueue<PutStatus> getQueue() {
471       return this.queue;
472     }
473 
474     public long getTotalFailedCount() {
475       return totalFailedPutCount.get();
476     }
477 
478     public long getTotalBufferedCount() {
479       return queue.size() + currentProcessingCount.get();
480     }
481 
482     public AtomicAverageCounter getAverageLatencyCounter() {
483       return this.averageLatency;
484     }
485 
486     public long getMaxLatency() {
487       return this.maxLatency.getAndSet(0);
488     }
489 
490     boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
491       // Decrease the retry count
492       final int retryCount = ps.maxAttempCount - 1;
493 
494       if (retryCount <= 0) {
495         // Update the failed counter and no retry any more.
496         return false;
497       }
498 
499       int cnt = getRetryInQueue().incrementAndGet();
500       if (cnt > getMaxRetryInQueue()) {
501         // Too many Puts in queue for resubmit, give up this
502         getRetryInQueue().decrementAndGet();
503         return false;
504       }
505 
506       final Put failedPut = ps.put;
507       // The currentPut is failed. So get the table name for the currentPut.
508       final TableName tableName = ps.regionInfo.getTable();
509 
510       long delayMs = getNextDelay(retryCount);
511       if (LOG.isDebugEnabled()) {
512         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
513       }
514 
515       getExecutor().schedule(new Runnable() {
516         @Override
517         public void run() {
518           boolean succ = false;
519           try {
520             succ = FlushWorker.this.getMultiplexer()._put(tableName, failedPut, retryCount, true);
521           } finally {
522             FlushWorker.this.getRetryInQueue().decrementAndGet();
523             if (!succ) {
524               FlushWorker.this.getTotalFailedPutCount().incrementAndGet();
525             }
526           }
527         }
528       }, delayMs, TimeUnit.MILLISECONDS);
529       return true;
530     }
531 
532     @VisibleForTesting
533     long getNextDelay(int retryCount) {
534       return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
535           multiplexer.maxAttempts - retryCount - 1);
536     }
537 
538     @VisibleForTesting
539     AtomicInteger getRetryInQueue() {
540       return this.retryInQueue;
541     }
542 
543     @VisibleForTesting
544     int getMaxRetryInQueue() {
545       return this.maxRetryInQueue;
546     }
547 
548     @VisibleForTesting
549     AtomicLong getTotalFailedPutCount() {
550       return this.totalFailedPutCount;
551     }
552 
553     @VisibleForTesting
554     HTableMultiplexer getMultiplexer() {
555       return this.multiplexer;
556     }
557 
558     @VisibleForTesting
559     ScheduledExecutorService getExecutor() {
560       return this.executor;
561     }
562 
563     @Override
564     public void run() {
565       int failedCount = 0;
566       try {
567         long start = EnvironmentEdgeManager.currentTime();
568 
569         // drain all the queued puts into the tmp list
570         processingList.clear();
571         queue.drainTo(processingList);
572         if (processingList.size() == 0) {
573           // Nothing to flush
574           return;
575         }
576 
577         currentProcessingCount.set(processingList.size());
578         // failedCount is decreased whenever a Put is success or resubmit.
579         failedCount = processingList.size();
580 
581         List<Action<Row>> retainedActions = new ArrayList<>(processingList.size());
582         MultiAction<Row> actions = new MultiAction<>();
583         for (int i = 0; i < processingList.size(); i++) {
584           PutStatus putStatus = processingList.get(i);
585           Action<Row> action = new Action<Row>(putStatus.put, i);
586           actions.add(putStatus.regionInfo.getRegionName(), action);
587           retainedActions.add(action);
588         }
589 
590         // Process this multi-put request
591         List<PutStatus> failed = null;
592         Object[] results = new Object[actions.size()];
593         ServerName server = addr.getServerName();
594         Map<ServerName, MultiAction<Row>> actionsByServer =
595             Collections.singletonMap(server, actions);
596         try {
597           AsyncRequestFuture arf =
598               ap.submitMultiActions(null, retainedActions, 0L, null, results, true, null,
599                 null, actionsByServer, null);
600           arf.waitUntilDone();
601           if (arf.hasError()) {
602             // We just log and ignore the exception here since failed Puts will be resubmit again.
603             LOG.debug("Caught some exceptions when flushing puts to region server "
604                 + addr.getHostnamePort(), arf.getErrors());
605           }
606         } finally {
607           for (int i = 0; i < results.length; i++) {
608             if (results[i] instanceof Result) {
609               failedCount--;
610             } else {
611               if (failed == null) {
612                 failed = new ArrayList<PutStatus>();
613               }
614               failed.add(processingList.get(i));
615             }
616           }
617         }
618 
619         if (failed != null) {
620           // Resubmit failed puts
621           for (PutStatus putStatus : failed) {
622             if (resubmitFailedPut(putStatus, this.addr)) {
623               failedCount--;
624             }
625           }
626         }
627 
628         long elapsed = EnvironmentEdgeManager.currentTime() - start;
629         // Update latency counters
630         averageLatency.add(elapsed);
631         if (elapsed > maxLatency.get()) {
632           maxLatency.set(elapsed);
633         }
634 
635         // Log some basic info
636         if (LOG.isDebugEnabled()) {
637           LOG.debug("Processed " + currentProcessingCount + " put requests for "
638               + addr.getHostnamePort() + " and " + failedCount + " failed"
639               + ", latency for this send: " + elapsed);
640         }
641 
642         // Reset the current processing put count
643         currentProcessingCount.set(0);
644       } catch (RuntimeException e) {
645         // To make findbugs happy
646         // Log all the exceptions and move on
647         LOG.debug(
648           "Caught some exceptions " + e + " when flushing puts to region server "
649               + addr.getHostnamePort(), e);
650       } catch (Exception e) {
651         if (e instanceof InterruptedException) {
652           Thread.currentThread().interrupt();
653         }
654         // Log all the exceptions and move on
655         LOG.debug(
656           "Caught some exceptions " + e + " when flushing puts to region server "
657               + addr.getHostnamePort(), e);
658       } finally {
659         // Update the totalFailedCount
660         this.totalFailedPutCount.addAndGet(failedCount);
661       }
662     }
663   }
664 }