View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase;
20  
21  import java.io.IOException;
22  import java.io.OutputStream;
23  import java.nio.ByteBuffer;
24  import java.util.ArrayList;
25  import java.util.List;
26  
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.KeyValue.Type;
29  import org.apache.hadoop.hbase.io.util.StreamUtils;
30  import org.apache.hadoop.hbase.util.ByteBufferUtils;
31  import org.apache.hadoop.hbase.util.Bytes;
32  import org.apache.hadoop.hbase.util.IterableUtils;
33  import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
34  import org.apache.hadoop.io.WritableUtils;
35  
36  import com.google.common.base.Function;
37  import com.google.common.collect.Lists;
38  
39  /**
40   * static convenience methods for dealing with KeyValues and collections of KeyValues
41   */
42  @InterfaceAudience.Private
43  public class KeyValueUtil {
44  
45    /**************** length *********************/
46  
47    /**
48     * Returns number of bytes this cell would have been used if serialized as in {@link KeyValue}
49     * @param cell
50     * @return the length
51     */
52    public static int length(final Cell cell) {
53      return length(cell.getRowLength(), cell.getFamilyLength(), cell.getQualifierLength(),
54          cell.getValueLength(), cell.getTagsLength(), true);
55    }
56  
57    private static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) {
58      if (withTags) {
59        return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen, tlen));
60      }
61      return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen));
62    }
63  
64    /**
65     * Returns number of bytes this cell's key part would have been used if serialized as in
66     * {@link KeyValue}. Key includes rowkey, family, qualifier, timestamp and type.
67     * @param cell
68     * @return the key length
69     */
70    public static int keyLength(final Cell cell) {
71      return keyLength(cell.getRowLength(), cell.getFamilyLength(), cell.getQualifierLength());
72    }
73  
74    private static int keyLength(short rlen, byte flen, int qlen) {
75      return (int) KeyValue.getKeyDataStructureSize(rlen, flen, qlen);
76    }
77  
78    public static int lengthWithMvccVersion(final KeyValue kv, final boolean includeMvccVersion) {
79      int length = kv.getLength();
80      if (includeMvccVersion) {
81        length += WritableUtils.getVIntSize(kv.getMvccVersion());
82      }
83      return length;
84    }
85  
86    public static int totalLengthWithMvccVersion(final Iterable<? extends KeyValue> kvs,
87        final boolean includeMvccVersion) {
88      int length = 0;
89      for (KeyValue kv : IterableUtils.nullSafe(kvs)) {
90        length += lengthWithMvccVersion(kv, includeMvccVersion);
91      }
92      return length;
93    }
94  
95  
96    /**************** copy key only *********************/
97  
98    public static KeyValue copyToNewKeyValue(final Cell cell) {
99      byte[] bytes = copyToNewByteArray(cell);
100     KeyValue kvCell = new KeyValue(bytes, 0, bytes.length);
101     kvCell.setSequenceId(cell.getMvccVersion());
102     return kvCell;
103   }
104 
105   /**
106    * The position will be set to the beginning of the new ByteBuffer
107    * @param cell
108    * @return the Bytebuffer containing the key part of the cell
109    */
110   public static ByteBuffer copyKeyToNewByteBuffer(final Cell cell) {
111     byte[] bytes = new byte[keyLength(cell)];
112     appendKeyTo(cell, bytes, 0);
113     ByteBuffer buffer = ByteBuffer.wrap(bytes);
114     return buffer;
115   }
116 
117   public static byte[] copyToNewByteArray(final Cell cell) {
118     int v1Length = length(cell);
119     byte[] backingBytes = new byte[v1Length];
120     appendToByteArray(cell, backingBytes, 0);
121     return backingBytes;
122   }
123 
124   public static int appendKeyTo(final Cell cell, final byte[] output,
125       final int offset) {
126     int nextOffset = offset;
127     nextOffset = Bytes.putShort(output, nextOffset, cell.getRowLength());
128     nextOffset = CellUtil.copyRowTo(cell, output, nextOffset);
129     nextOffset = Bytes.putByte(output, nextOffset, cell.getFamilyLength());
130     nextOffset = CellUtil.copyFamilyTo(cell, output, nextOffset);
131     nextOffset = CellUtil.copyQualifierTo(cell, output, nextOffset);
132     nextOffset = Bytes.putLong(output, nextOffset, cell.getTimestamp());
133     nextOffset = Bytes.putByte(output, nextOffset, cell.getTypeByte());
134     return nextOffset;
135   }
136 
137 
138   /**************** copy key and value *********************/
139 
140   public static int appendToByteArray(final Cell cell, final byte[] output, final int offset) {
141     // TODO when cell instance of KV we can bypass all steps and just do backing single array
142     // copy(?)
143     int pos = offset;
144     pos = Bytes.putInt(output, pos, keyLength(cell));
145     pos = Bytes.putInt(output, pos, cell.getValueLength());
146     pos = appendKeyTo(cell, output, pos);
147     pos = CellUtil.copyValueTo(cell, output, pos);
148     if ((cell.getTagsLength() > 0)) {
149       pos = Bytes.putAsShort(output, pos, cell.getTagsLength());
150       pos = CellUtil.copyTagTo(cell, output, pos);
151     }
152     return pos;
153   }
154 
155   /**
156    * The position will be set to the beginning of the new ByteBuffer
157    * @param cell
158    * @return the ByteBuffer containing the cell
159    */
160   public static ByteBuffer copyToNewByteBuffer(final Cell cell) {
161     byte[] bytes = new byte[length(cell)];
162     appendToByteArray(cell, bytes, 0);
163     ByteBuffer buffer = ByteBuffer.wrap(bytes);
164     return buffer;
165   }
166 
167   public static void appendToByteBuffer(final ByteBuffer bb, final KeyValue kv,
168       final boolean includeMvccVersion) {
169     // keep pushing the limit out. assume enough capacity
170     bb.limit(bb.position() + kv.getLength());
171     bb.put(kv.getBuffer(), kv.getOffset(), kv.getLength());
172     if (includeMvccVersion) {
173       int numMvccVersionBytes = WritableUtils.getVIntSize(kv.getMvccVersion());
174       ByteBufferUtils.extendLimit(bb, numMvccVersionBytes);
175       ByteBufferUtils.writeVLong(bb, kv.getMvccVersion());
176     }
177   }
178 
179 
180   /**************** iterating *******************************/
181 
182   /**
183    * Creates a new KeyValue object positioned in the supplied ByteBuffer and sets the ByteBuffer's
184    * position to the start of the next KeyValue. Does not allocate a new array or copy data.
185    * @param bb
186    * @param includesMvccVersion
187    * @param includesTags
188    */
189   public static KeyValue nextShallowCopy(final ByteBuffer bb, final boolean includesMvccVersion,
190       boolean includesTags) {
191     if (bb.isDirect()) {
192       throw new IllegalArgumentException("only supports heap buffers");
193     }
194     if (bb.remaining() < 1) {
195       return null;
196     }
197     KeyValue keyValue = null;
198     int underlyingArrayOffset = bb.arrayOffset() + bb.position();
199     int keyLength = bb.getInt();
200     int valueLength = bb.getInt();
201     ByteBufferUtils.skip(bb, keyLength + valueLength);
202     int tagsLength = 0;
203     if (includesTags) {
204       // Read short as unsigned, high byte first
205       tagsLength = ((bb.get() & 0xff) << 8) ^ (bb.get() & 0xff);
206       ByteBufferUtils.skip(bb, tagsLength);
207     }
208     int kvLength = (int) KeyValue.getKeyValueDataStructureSize(keyLength, valueLength, tagsLength);
209     keyValue = new KeyValue(bb.array(), underlyingArrayOffset, kvLength);
210     if (includesMvccVersion) {
211       long mvccVersion = ByteBufferUtils.readVLong(bb);
212       keyValue.setSequenceId(mvccVersion);
213     }
214     return keyValue;
215   }
216 
217 
218   /*************** next/previous **********************************/
219 
220   /**
221    * Append single byte 0x00 to the end of the input row key
222    */
223   public static KeyValue createFirstKeyInNextRow(final Cell in){
224     byte[] nextRow = new byte[in.getRowLength() + 1];
225     System.arraycopy(in.getRowArray(), in.getRowOffset(), nextRow, 0, in.getRowLength());
226     nextRow[nextRow.length - 1] = 0;//maybe not necessary
227     return createFirstOnRow(nextRow);
228   }
229 
230   /**
231    * Increment the row bytes and clear the other fields
232    */
233   public static KeyValue createFirstKeyInIncrementedRow(final Cell in){
234     byte[] thisRow = new SimpleMutableByteRange(in.getRowArray(), in.getRowOffset(),
235         in.getRowLength()).deepCopyToNewArray();
236     byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow);
237     return createFirstOnRow(nextRow);
238   }
239 
240   /**
241    * Decrement the timestamp.  For tests (currently wasteful)
242    *
243    * Remember timestamps are sorted reverse chronologically.
244    * @param in
245    * @return previous key
246    */
247   public static KeyValue previousKey(final KeyValue in) {
248     return createFirstOnRow(CellUtil.cloneRow(in), CellUtil.cloneFamily(in),
249       CellUtil.cloneQualifier(in), in.getTimestamp() - 1);
250   }
251 
252 
253   /**
254    * Create a KeyValue for the specified row, family and qualifier that would be
255    * larger than or equal to all other possible KeyValues that have the same
256    * row, family, qualifier. Used for reseeking.
257    *
258    * @param row
259    *          row key
260    * @param roffset
261    *         row offset
262    * @param rlength
263    *         row length
264    * @param family
265    *         family name
266    * @param foffset
267    *         family offset
268    * @param flength
269    *         family length
270    * @param qualifier
271    *        column qualifier
272    * @param qoffset
273    *        qualifier offset
274    * @param qlength
275    *        qualifier length
276    * @return Last possible key on passed row, family, qualifier.
277    */
278   public static KeyValue createLastOnRow(final byte[] row, final int roffset, final int rlength,
279       final byte[] family, final int foffset, final int flength, final byte[] qualifier,
280       final int qoffset, final int qlength) {
281     return new KeyValue(row, roffset, rlength, family, foffset, flength, qualifier, qoffset,
282         qlength, HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
283   }
284   
285   /**
286    * Creates a keyValue for the specified keyvalue larger than or equal to all other possible
287    * KeyValues that have the same row, family, qualifer.  Used for reseeking
288    * @param kv
289    * @return KeyValue
290    */
291   public static KeyValue createLastOnRow(Cell kv) {
292     return createLastOnRow(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), null, 0, 0,
293         null, 0, 0);
294   }
295 
296   /**
297    * Similar to
298    * {@link #createLastOnRow(byte[], int, int, byte[], int, int, byte[], int, int)}
299    * but creates the last key on the row/column of this KV (the value part of
300    * the returned KV is always empty). Used in creating "fake keys" for the
301    * multi-column Bloom filter optimization to skip the row/column we already
302    * know is not in the file.
303    * 
304    * @param kv - cell
305    * @return the last key on the row/column of the given key-value pair
306    */
307   public static KeyValue createLastOnRowCol(Cell kv) {
308     return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
309         kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
310         kv.getQualifierOffset(), kv.getQualifierLength(), HConstants.OLDEST_TIMESTAMP,
311         Type.Minimum, null, 0, 0);
312   }
313 
314   /**
315    * Creates the first KV with the row/family/qualifier of this KV and the given
316    * timestamp. Uses the "maximum" KV type that guarantees that the new KV is
317    * the lowest possible for this combination of row, family, qualifier, and
318    * timestamp. This KV's own timestamp is ignored. While this function copies
319    * the value from this KV, it is normally used on key-only KVs.
320    * 
321    * @param kv - cell
322    * @param ts
323    */
324   public static KeyValue createFirstOnRowColTS(Cell kv, long ts) {
325     return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
326         kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
327         kv.getQualifierOffset(), kv.getQualifierLength(), ts, Type.Maximum, kv.getValueArray(),
328         kv.getValueOffset(), kv.getValueLength());
329   }
330   
331   /**
332    * Create a KeyValue that is smaller than all other possible KeyValues
333    * for the given row. That is any (valid) KeyValue on 'row' would sort
334    * _after_ the result.
335    *
336    * @param row - row key (arbitrary byte array)
337    * @return First possible KeyValue on passed <code>row</code>
338    */
339   public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
340     return new KeyValue(row, roffset, rlength,
341         null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
342   }
343   
344 
345   /**
346    * Creates a KeyValue that is last on the specified row id. That is,
347    * every other possible KeyValue for the given row would compareTo()
348    * less than the result of this call.
349    * @param row row key
350    * @return Last possible KeyValue on passed <code>row</code>
351    */
352   public static KeyValue createLastOnRow(final byte[] row) {
353     return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
354   }
355 
356   /**
357    * Create a KeyValue that is smaller than all other possible KeyValues
358    * for the given row. That is any (valid) KeyValue on 'row' would sort
359    * _after_ the result.
360    *
361    * @param row - row key (arbitrary byte array)
362    * @return First possible KeyValue on passed <code>row</code>
363    */
364   public static KeyValue createFirstOnRow(final byte [] row) {
365     return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
366   }
367 
368   /**
369    * Creates a KeyValue that is smaller than all other KeyValues that
370    * are older than the passed timestamp.
371    * @param row - row key (arbitrary byte array)
372    * @param ts - timestamp
373    * @return First possible key on passed <code>row</code> and timestamp.
374    */
375   public static KeyValue createFirstOnRow(final byte [] row,
376       final long ts) {
377     return new KeyValue(row, null, null, ts, Type.Maximum);
378   }
379 
380   /**
381    * Create a KeyValue for the specified row, family and qualifier that would be
382    * smaller than all other possible KeyValues that have the same row,family,qualifier.
383    * Used for seeking.
384    * @param row - row key (arbitrary byte array)
385    * @param family - family name
386    * @param qualifier - column qualifier
387    * @return First possible key on passed <code>row</code>, and column.
388    */
389   public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
390       final byte [] qualifier) {
391     return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
392   }
393 
394   /**
395    * Create a Delete Family KeyValue for the specified row and family that would
396    * be smaller than all other possible Delete Family KeyValues that have the
397    * same row and family.
398    * Used for seeking.
399    * @param row - row key (arbitrary byte array)
400    * @param family - family name
401    * @return First Delete Family possible key on passed <code>row</code>.
402    */
403   public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
404       final byte [] family) {
405     return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
406         Type.DeleteFamily);
407   }
408 
409   /**
410    * @param row - row key (arbitrary byte array)
411    * @param f - family name
412    * @param q - column qualifier
413    * @param ts - timestamp
414    * @return First possible key on passed <code>row</code>, column and timestamp
415    */
416   public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
417       final byte [] q, final long ts) {
418     return new KeyValue(row, f, q, ts, Type.Maximum);
419   }
420 
421   /**
422    * Create a KeyValue for the specified row, family and qualifier that would be
423    * smaller than all other possible KeyValues that have the same row,
424    * family, qualifier.
425    * Used for seeking.
426    * @param row row key
427    * @param roffset row offset
428    * @param rlength row length
429    * @param family family name
430    * @param foffset family offset
431    * @param flength family length
432    * @param qualifier column qualifier
433    * @param qoffset qualifier offset
434    * @param qlength qualifier length
435    * @return First possible key on passed Row, Family, Qualifier.
436    */
437   public static KeyValue createFirstOnRow(final byte [] row,
438       final int roffset, final int rlength, final byte [] family,
439       final int foffset, final int flength, final byte [] qualifier,
440       final int qoffset, final int qlength) {
441     return new KeyValue(row, roffset, rlength, family,
442         foffset, flength, qualifier, qoffset, qlength,
443         HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
444   }
445 
446   /**
447    * Create a KeyValue for the specified row, family and qualifier that would be
448    * smaller than all other possible KeyValues that have the same row,
449    * family, qualifier.
450    * Used for seeking.
451    *
452    * @param buffer the buffer to use for the new <code>KeyValue</code> object
453    * @param row the value key
454    * @param family family name
455    * @param qualifier column qualifier
456    *
457    * @return First possible key on passed Row, Family, Qualifier.
458    *
459    * @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
460    * than the provided buffer or than <code>Integer.MAX_VALUE</code>
461    */
462   public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
463       final byte [] family, final byte [] qualifier)
464           throws IllegalArgumentException {
465     return createFirstOnRow(buffer, 0, row, 0, row.length,
466         family, 0, family.length,
467         qualifier, 0, qualifier.length);
468   }
469 
470   /**
471    * Create a KeyValue for the specified row, family and qualifier that would be
472    * smaller than all other possible KeyValues that have the same row,
473    * family, qualifier.
474    * Used for seeking.
475    *
476    * @param buffer the buffer to use for the new <code>KeyValue</code> object
477    * @param boffset buffer offset
478    * @param row the value key
479    * @param roffset row offset
480    * @param rlength row length
481    * @param family family name
482    * @param foffset family offset
483    * @param flength family length
484    * @param qualifier column qualifier
485    * @param qoffset qualifier offset
486    * @param qlength qualifier length
487    *
488    * @return First possible key on passed Row, Family, Qualifier.
489    *
490    * @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
491    * than the provided buffer or than <code>Integer.MAX_VALUE</code>
492    */
493   public static KeyValue createFirstOnRow(byte[] buffer, final int boffset, final byte[] row,
494       final int roffset, final int rlength, final byte[] family, final int foffset,
495       final int flength, final byte[] qualifier, final int qoffset, final int qlength)
496       throws IllegalArgumentException {
497 
498     long lLength = KeyValue.getKeyValueDataStructureSize(rlength, flength, qlength, 0);
499 
500     if (lLength > Integer.MAX_VALUE) {
501       throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
502     }
503     int iLength = (int) lLength;
504     if (buffer.length - boffset < iLength) {
505       throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < "
506           + iLength);
507     }
508 
509     int len = KeyValue.writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset,
510         flength, qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
511         null, 0, 0, null);
512     return new KeyValue(buffer, boffset, len);
513   }
514 
515   /**
516    * Creates the first KV with the row/family/qualifier of this KV and the
517    * given timestamp. Uses the "maximum" KV type that guarantees that the new
518    * KV is the lowest possible for this combination of row, family, qualifier,
519    * and timestamp. This KV's own timestamp is ignored. While this function
520    * copies the value from this KV, it is normally used on key-only KVs.
521    */
522   public static KeyValue createFirstOnRowColTS(KeyValue kv, long ts) {
523     return new KeyValue(
524         kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
525         kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
526         kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
527         ts, Type.Maximum, kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
528   }
529 
530   /*************** misc **********************************/
531   /**
532    * @param cell
533    * @return <code>cell<code> if it is an instance of {@link KeyValue} else we will return a
534    * new {@link KeyValue} instance made from <code>cell</code>
535    * @deprecated without any replacement.
536    */
537   @Deprecated
538   public static KeyValue ensureKeyValue(final Cell cell) {
539     if (cell == null) return null;
540     return cell instanceof KeyValue? (KeyValue)cell: copyToNewKeyValue(cell);
541   }
542 
543   @Deprecated
544   public static List<KeyValue> ensureKeyValues(List<Cell> cells) {
545     List<KeyValue> lazyList = Lists.transform(cells, new Function<Cell, KeyValue>() {
546       @Override
547       public KeyValue apply(Cell arg0) {
548         return KeyValueUtil.ensureKeyValue(arg0);
549       }
550     });
551     return new ArrayList<KeyValue>(lazyList);
552   }
553 
554   public static void oswrite(final Cell cell, final OutputStream out, final boolean withTags)
555       throws IOException {
556     if (cell instanceof KeyValue) {
557       KeyValue.oswrite((KeyValue) cell, out, withTags);
558     } else {
559       short rlen = cell.getRowLength();
560       byte flen = cell.getFamilyLength();
561       int qlen = cell.getQualifierLength();
562       int vlen = cell.getValueLength();
563       int tlen = cell.getTagsLength();
564 
565       // write total length
566       StreamUtils.writeInt(out, length(rlen, flen, qlen, vlen, tlen, withTags));
567       // write key length
568       StreamUtils.writeInt(out, keyLength(rlen, flen, qlen));
569       // write value length
570       StreamUtils.writeInt(out, vlen);
571       // Write rowkey - 2 bytes rk length followed by rowkey bytes
572       StreamUtils.writeShort(out, rlen);
573       out.write(cell.getRowArray(), cell.getRowOffset(), rlen);
574       // Write cf - 1 byte of cf length followed by the family bytes
575       out.write(flen);
576       out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flen);
577       // write qualifier
578       out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
579       // write timestamp
580       StreamUtils.writeLong(out, cell.getTimestamp());
581       // write the type
582       out.write(cell.getTypeByte());
583       // write value
584       out.write(cell.getValueArray(), cell.getValueOffset(), vlen);
585       // write tags if we have to
586       if (withTags && tlen > 0) {
587         // 2 bytes tags length followed by tags bytes
588         // tags length is serialized with 2 bytes only(short way) even if the type is int. As this
589         // is non -ve numbers, we save the sign bit. See HBASE-11437
590         out.write((byte) (0xff & (tlen >> 8)));
591         out.write((byte) (0xff & tlen));
592         out.write(cell.getTagsArray(), cell.getTagsOffset(), tlen);
593       }
594     }
595   }
596 }