View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.rest.client;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.Iterator;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.Set;
29  import java.util.TreeMap;
30  
31  import com.google.protobuf.Service;
32  import com.google.protobuf.ServiceException;
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.client.*;
38  import org.apache.hadoop.hbase.client.coprocessor.Batch;
39  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
40  import org.apache.hadoop.util.StringUtils;
41  
42  import org.apache.hadoop.classification.InterfaceAudience;
43  import org.apache.hadoop.classification.InterfaceStability;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.hbase.Cell;
46  import org.apache.hadoop.hbase.HBaseConfiguration;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.HTableDescriptor;
49  import org.apache.hadoop.hbase.KeyValue;
50  import org.apache.hadoop.hbase.KeyValueUtil;
51  import org.apache.hadoop.hbase.client.RowMutations;
52  import org.apache.hadoop.hbase.client.Delete;
53  import org.apache.hadoop.hbase.client.Get;
54  import org.apache.hadoop.hbase.client.HTableInterface;
55  import org.apache.hadoop.hbase.client.Increment;
56  import org.apache.hadoop.hbase.client.Put;
57  import org.apache.hadoop.hbase.client.Row;
58  import org.apache.hadoop.hbase.client.Result;
59  import org.apache.hadoop.hbase.client.ResultScanner;
60  import org.apache.hadoop.hbase.client.Scan;
61  import org.apache.hadoop.hbase.io.TimeRange;
62  import org.apache.hadoop.hbase.rest.Constants;
63  import org.apache.hadoop.hbase.rest.model.CellModel;
64  import org.apache.hadoop.hbase.rest.model.CellSetModel;
65  import org.apache.hadoop.hbase.rest.model.RowModel;
66  import org.apache.hadoop.hbase.rest.model.ScannerModel;
67  import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
68  import org.apache.hadoop.hbase.util.Bytes;
69  
70  /**
71   * HTable interface to remote tables accessed via REST gateway
72   */
73  @InterfaceAudience.Public
74  @InterfaceStability.Stable
75  public class RemoteHTable implements HTableInterface {
76  
77    private static final Log LOG = LogFactory.getLog(RemoteHTable.class);
78  
79    final Client client;
80    final Configuration conf;
81    final byte[] name;
82    final int maxRetries;
83    final long sleepTime;
84  
85    @SuppressWarnings("rawtypes")
86    protected String buildRowSpec(final byte[] row, final Map familyMap,
87        final long startTime, final long endTime, final int maxVersions) {
88      StringBuffer sb = new StringBuffer();
89      sb.append('/');
90      sb.append(Bytes.toStringBinary(name));
91      sb.append('/');
92      sb.append(Bytes.toStringBinary(row));
93      Set families = familyMap.entrySet();
94      if (families != null) {
95        Iterator i = familyMap.entrySet().iterator();
96        sb.append('/');
97        while (i.hasNext()) {
98          Map.Entry e = (Map.Entry)i.next();
99          Collection quals = (Collection)e.getValue();
100         if (quals != null && !quals.isEmpty()) {
101           Iterator ii = quals.iterator();
102           while (ii.hasNext()) {
103             sb.append(Bytes.toStringBinary((byte[])e.getKey()));
104             sb.append(':');
105             Object o = ii.next();
106             // Puts use byte[] but Deletes use KeyValue
107             if (o instanceof byte[]) {
108               sb.append(Bytes.toStringBinary((byte[])o));
109             } else if (o instanceof KeyValue) {
110               sb.append(Bytes.toStringBinary(((KeyValue)o).getQualifier()));
111             } else {
112               throw new RuntimeException("object type not handled");
113             }
114             if (ii.hasNext()) {
115               sb.append(',');
116             }
117           }
118         } else {
119           sb.append(Bytes.toStringBinary((byte[])e.getKey()));
120           sb.append(':');
121         }
122         if (i.hasNext()) {
123           sb.append(',');
124         }
125       }
126     }
127     if (startTime != 0 && endTime != Long.MAX_VALUE) {
128       sb.append('/');
129       sb.append(startTime);
130       if (startTime != endTime) {
131         sb.append(',');
132         sb.append(endTime);
133       }
134     } else if (endTime != Long.MAX_VALUE) {
135       sb.append('/');
136       sb.append(endTime);
137     }
138     if (maxVersions > 1) {
139       sb.append("?v=");
140       sb.append(maxVersions);
141     }
142     return sb.toString();
143   }
144 
145   protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
146     StringBuilder sb = new StringBuilder();
147     sb.append('/');
148     sb.append(Bytes.toStringBinary(name));
149     sb.append("/multiget/");
150     if (rows == null || rows.length == 0) {
151       return sb.toString();
152     }
153     sb.append("?");
154     for(int i=0; i<rows.length; i++) {
155       byte[] rk = rows[i];
156       if (i != 0) {
157         sb.append('&');
158       }
159       sb.append("row=");
160       sb.append(Bytes.toStringBinary(rk));
161     }
162     sb.append("&v=");
163     sb.append(maxVersions);
164 
165     return sb.toString();
166   }
167 
168   protected Result[] buildResultFromModel(final CellSetModel model) {
169     List<Result> results = new ArrayList<Result>();
170     for (RowModel row: model.getRows()) {
171       List<KeyValue> kvs = new ArrayList<KeyValue>();
172       for (CellModel cell: row.getCells()) {
173         byte[][] split = KeyValue.parseColumn(cell.getColumn());
174         byte[] column = split[0];
175         byte[] qualifier = split.length > 1 ? split[1] : null;
176         kvs.add(new KeyValue(row.getKey(), column, qualifier,
177           cell.getTimestamp(), cell.getValue()));
178       }
179       results.add(new Result(kvs));
180     }
181     return results.toArray(new Result[results.size()]);
182   }
183 
184   protected CellSetModel buildModelFromPut(Put put) {
185     RowModel row = new RowModel(put.getRow());
186     long ts = put.getTimeStamp();
187     for (List<Cell> cells: put.getFamilyCellMap().values()) {
188       for (Cell cell: cells) {
189         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
190         row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
191           ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(),
192           kv.getValue()));
193       }
194     }
195     CellSetModel model = new CellSetModel();
196     model.addRow(row);
197     return model;
198   }
199 
200   /**
201    * Constructor
202    * @param client
203    * @param name
204    */
205   public RemoteHTable(Client client, String name) {
206     this(client, HBaseConfiguration.create(), Bytes.toBytes(name));
207   }
208 
209   /**
210    * Constructor
211    * @param client
212    * @param conf
213    * @param name
214    */
215   public RemoteHTable(Client client, Configuration conf, String name) {
216     this(client, conf, Bytes.toBytes(name));
217   }
218 
219   /**
220    * Constructor
221    * @param client
222    * @param conf
223    * @param name
224    */
225   public RemoteHTable(Client client, Configuration conf, byte[] name) {
226     this.client = client;
227     this.conf = conf;
228     this.name = name;
229     this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
230     this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
231   }
232 
233   public byte[] getTableName() {
234     return name.clone();
235   }
236 
237   @Override
238   public TableName getName() {
239     return TableName.valueOf(name);
240   }
241 
242   public Configuration getConfiguration() {
243     return conf;
244   }
245 
246   public HTableDescriptor getTableDescriptor() throws IOException {
247     StringBuilder sb = new StringBuilder();
248     sb.append('/');
249     sb.append(Bytes.toStringBinary(name));
250     sb.append('/');
251     sb.append("schema");
252     for (int i = 0; i < maxRetries; i++) {
253       Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
254       int code = response.getCode();
255       switch (code) {
256       case 200:
257         TableSchemaModel schema = new TableSchemaModel();
258         schema.getObjectFromMessage(response.getBody());
259         return schema.getTableDescriptor();
260       case 509:
261         try {
262           Thread.sleep(sleepTime);
263         } catch (InterruptedException e) { }
264         break;
265       default:
266         throw new IOException("schema request returned " + code);
267       }
268     }
269     throw new IOException("schema request timed out");
270   }
271 
272   public void close() throws IOException {
273     client.shutdown();
274   }
275 
276   public Result get(Get get) throws IOException {
277     TimeRange range = get.getTimeRange();
278     String spec = buildRowSpec(get.getRow(), get.getFamilyMap(),
279       range.getMin(), range.getMax(), get.getMaxVersions());
280     if (get.getFilter() != null) {
281       LOG.warn("filters not supported on gets");
282     }
283     Result[] results = getResults(spec);
284     if (results.length > 0) {
285       if (results.length > 1) {
286         LOG.warn("too many results for get (" + results.length + ")");
287       }
288       return results[0];
289     } else {
290       return new Result();
291     }
292   }
293 
294   public Result[] get(List<Get> gets) throws IOException {
295     byte[][] rows = new byte[gets.size()][];
296     int maxVersions = 1;
297     int count = 0;
298 
299     for(Get g:gets) {
300 
301       if ( count == 0 ) {
302         maxVersions = g.getMaxVersions();
303       } else if (g.getMaxVersions() != maxVersions) {
304         LOG.warn("MaxVersions on Gets do not match, using the first in the list ("+maxVersions+")");
305       }
306 
307       if (g.getFilter() != null) {
308         LOG.warn("filters not supported on gets");
309       }
310 
311       rows[count] = g.getRow();
312       count ++;
313     }
314 
315     String spec = buildMultiRowSpec(rows, maxVersions);
316 
317     return getResults(spec);
318   }
319 
320   private Result[] getResults(String spec) throws IOException {
321     for (int i = 0; i < maxRetries; i++) {
322       Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
323       int code = response.getCode();
324       switch (code) {
325         case 200:
326           CellSetModel model = new CellSetModel();
327           model.getObjectFromMessage(response.getBody());
328           Result[] results = buildResultFromModel(model);
329           if ( results.length > 0) {
330             return results;
331           }
332           // fall through
333         case 404:
334           return new Result[0];
335 
336         case 509:
337           try {
338             Thread.sleep(sleepTime);
339           } catch (InterruptedException e) { }
340           break;
341         default:
342           throw new IOException("get request returned " + code);
343       }
344     }
345     throw new IOException("get request timed out");
346   }
347 
348   public boolean exists(Get get) throws IOException {
349     LOG.warn("exists() is really get(), just use get()");
350     Result result = get(get);
351     return (result != null && !(result.isEmpty()));
352   }
353 
354   /**
355    * exists(List) is really a list of get() calls. Just use get().
356    * @param gets list of Get to test for the existence
357    */
358   public Boolean[] exists(List<Get> gets) throws IOException {
359     LOG.warn("exists(List<Get>) is really list of get() calls, just use get()");
360     Boolean[] results = new Boolean[gets.size()];
361     for (int i = 0; i < results.length; i++) {
362       results[i] = exists(gets.get(i));
363     }
364     return results;
365   }
366 
367   public void put(Put put) throws IOException {
368     CellSetModel model = buildModelFromPut(put);
369     StringBuilder sb = new StringBuilder();
370     sb.append('/');
371     sb.append(Bytes.toStringBinary(name));
372     sb.append('/');
373     sb.append(Bytes.toStringBinary(put.getRow()));
374     for (int i = 0; i < maxRetries; i++) {
375       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
376         model.createProtobufOutput());
377       int code = response.getCode();
378       switch (code) {
379       case 200:
380         return;
381       case 509:
382         try {
383           Thread.sleep(sleepTime);
384         } catch (InterruptedException e) { }
385         break;
386       default:
387         throw new IOException("put request failed with " + code);
388       }
389     }
390     throw new IOException("put request timed out");
391   }
392 
393   public void put(List<Put> puts) throws IOException {
394     // this is a trick: The gateway accepts multiple rows in a cell set and
395     // ignores the row specification in the URI
396 
397     // separate puts by row
398     TreeMap<byte[],List<Cell>> map =
399       new TreeMap<byte[],List<Cell>>(Bytes.BYTES_COMPARATOR);
400     for (Put put: puts) {
401       byte[] row = put.getRow();
402       List<Cell> cells = map.get(row);
403       if (cells == null) {
404         cells = new ArrayList<Cell>();
405         map.put(row, cells);
406       }
407       for (List<Cell> l: put.getFamilyCellMap().values()) {
408         cells.addAll(l);
409       }
410     }
411 
412     // build the cell set
413     CellSetModel model = new CellSetModel();
414     for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) {
415       RowModel row = new RowModel(e.getKey());
416       for (Cell cell: e.getValue()) {
417         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
418         row.addCell(new CellModel(kv));
419       }
420       model.addRow(row);
421     }
422 
423     // build path for multiput
424     StringBuilder sb = new StringBuilder();
425     sb.append('/');
426     sb.append(Bytes.toStringBinary(name));
427     sb.append("/$multiput"); // can be any nonexistent row
428     for (int i = 0; i < maxRetries; i++) {
429       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
430         model.createProtobufOutput());
431       int code = response.getCode();
432       switch (code) {
433       case 200:
434         return;
435       case 509:
436         try {
437           Thread.sleep(sleepTime);
438         } catch (InterruptedException e) { }
439         break;
440       default:
441         throw new IOException("multiput request failed with " + code);
442       }
443     }
444     throw new IOException("multiput request timed out");
445   }
446 
447   public void delete(Delete delete) throws IOException {
448     String spec = buildRowSpec(delete.getRow(), delete.getFamilyCellMap(),
449       delete.getTimeStamp(), delete.getTimeStamp(), 1);
450     for (int i = 0; i < maxRetries; i++) {
451       Response response = client.delete(spec);
452       int code = response.getCode();
453       switch (code) {
454       case 200:
455         return;
456       case 509:
457         try {
458           Thread.sleep(sleepTime);
459         } catch (InterruptedException e) { }
460         break;
461       default:
462         throw new IOException("delete request failed with " + code);
463       }
464     }
465     throw new IOException("delete request timed out");
466   }
467 
468   public void delete(List<Delete> deletes) throws IOException {
469     for (Delete delete: deletes) {
470       delete(delete);
471     }
472   }
473 
474   public void flushCommits() throws IOException {
475     // no-op
476   }
477 
478   class Scanner implements ResultScanner {
479 
480     String uri;
481 
482     public Scanner(Scan scan) throws IOException {
483       ScannerModel model;
484       try {
485         model = ScannerModel.fromScan(scan);
486       } catch (Exception e) {
487         throw new IOException(e);
488       }
489       StringBuffer sb = new StringBuffer();
490       sb.append('/');
491       sb.append(Bytes.toStringBinary(name));
492       sb.append('/');
493       sb.append("scanner");
494       for (int i = 0; i < maxRetries; i++) {
495         Response response = client.post(sb.toString(),
496           Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
497         int code = response.getCode();
498         switch (code) {
499         case 201:
500           uri = response.getLocation();
501           return;
502         case 509:
503           try {
504             Thread.sleep(sleepTime);
505           } catch (InterruptedException e) { }
506           break;
507         default:
508           throw new IOException("scan request failed with " + code);
509         }
510       }
511       throw new IOException("scan request timed out");
512     }
513 
514     @Override
515     public Result[] next(int nbRows) throws IOException {
516       StringBuilder sb = new StringBuilder(uri);
517       sb.append("?n=");
518       sb.append(nbRows);
519       for (int i = 0; i < maxRetries; i++) {
520         Response response = client.get(sb.toString(),
521           Constants.MIMETYPE_PROTOBUF);
522         int code = response.getCode();
523         switch (code) {
524         case 200:
525           CellSetModel model = new CellSetModel();
526           model.getObjectFromMessage(response.getBody());
527           return buildResultFromModel(model);
528         case 204:
529         case 206:
530           return null;
531         case 509:
532           try {
533             Thread.sleep(sleepTime);
534           } catch (InterruptedException e) { }
535           break;
536         default:
537           throw new IOException("scanner.next request failed with " + code);
538         }
539       }
540       throw new IOException("scanner.next request timed out");
541     }
542 
543     @Override
544     public Result next() throws IOException {
545       Result[] results = next(1);
546       if (results == null || results.length < 1) {
547         return null;
548       }
549       return results[0];
550     }
551 
552     class Iter implements Iterator<Result> {
553 
554       Result cache;
555 
556       public Iter() {
557         try {
558           cache = Scanner.this.next();
559         } catch (IOException e) {
560           LOG.warn(StringUtils.stringifyException(e));
561         }
562       }
563 
564       @Override
565       public boolean hasNext() {
566         return cache != null;
567       }
568 
569       @Override
570       public Result next() {
571         Result result = cache;
572         try {
573           cache = Scanner.this.next();
574         } catch (IOException e) {
575           LOG.warn(StringUtils.stringifyException(e));
576           cache = null;
577         }
578         return result;
579       }
580 
581       @Override
582       public void remove() {
583         throw new RuntimeException("remove() not supported");
584       }
585 
586     }
587 
588     @Override
589     public Iterator<Result> iterator() {
590       return new Iter();
591     }
592 
593     @Override
594     public void close() {
595       try {
596         client.delete(uri);
597       } catch (IOException e) {
598         LOG.warn(StringUtils.stringifyException(e));
599       }
600     }
601 
602   }
603 
604   public ResultScanner getScanner(Scan scan) throws IOException {
605     return new Scanner(scan);
606   }
607 
608   public ResultScanner getScanner(byte[] family) throws IOException {
609     Scan scan = new Scan();
610     scan.addFamily(family);
611     return new Scanner(scan);
612   }
613 
614   public ResultScanner getScanner(byte[] family, byte[] qualifier)
615       throws IOException {
616     Scan scan = new Scan();
617     scan.addColumn(family, qualifier);
618     return new Scanner(scan);
619   }
620 
621   public boolean isAutoFlush() {
622     return true;
623   }
624 
625   public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
626     throw new IOException("getRowOrBefore not supported");
627   }
628 
629   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
630       byte[] value, Put put) throws IOException {
631     // column to check-the-value
632     put.add(new KeyValue(row, family, qualifier, value));
633 
634     CellSetModel model = buildModelFromPut(put);
635     StringBuilder sb = new StringBuilder();
636     sb.append('/');
637     sb.append(Bytes.toStringBinary(name));
638     sb.append('/');
639     sb.append(Bytes.toStringBinary(put.getRow()));
640     sb.append("?check=put");
641 
642     for (int i = 0; i < maxRetries; i++) {
643       Response response = client.put(sb.toString(),
644         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
645       int code = response.getCode();
646       switch (code) {
647       case 200:
648         return true;
649       case 304: // NOT-MODIFIED
650         return false;
651       case 509:
652         try {
653           Thread.sleep(sleepTime);
654         } catch (final InterruptedException e) {
655         }
656         break;
657       default:
658         throw new IOException("checkAndPut request failed with " + code);
659       }
660     }
661     throw new IOException("checkAndPut request timed out");
662   }
663 
664   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
665       byte[] value, Delete delete) throws IOException {
666     Put put = new Put(row);
667     // column to check-the-value
668     put.add(new KeyValue(row, family, qualifier, value));
669     CellSetModel model = buildModelFromPut(put);
670     StringBuilder sb = new StringBuilder();
671     sb.append('/');
672     sb.append(Bytes.toStringBinary(name));
673     sb.append('/');
674     sb.append(Bytes.toStringBinary(row));
675     sb.append("?check=delete");
676 
677     for (int i = 0; i < maxRetries; i++) {
678       Response response = client.put(sb.toString(),
679         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
680       int code = response.getCode();
681       switch (code) {
682       case 200:
683         return true;
684       case 304: // NOT-MODIFIED
685         return false;
686       case 509:
687         try {
688           Thread.sleep(sleepTime);
689         } catch (final InterruptedException e) {
690         }
691         break;
692       default:
693         throw new IOException("checkAndDelete request failed with " + code);
694       }
695     }
696     throw new IOException("checkAndDelete request timed out");
697   }
698 
699   public Result increment(Increment increment) throws IOException {
700     throw new IOException("Increment not supported");
701   }
702 
703   public Result append(Append append) throws IOException {
704     throw new IOException("Append not supported");
705   }
706 
707   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
708       long amount) throws IOException {
709     throw new IOException("incrementColumnValue not supported");
710   }
711 
712   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
713       long amount, Durability durability) throws IOException {
714     throw new IOException("incrementColumnValue not supported");
715   }
716 
717   @Override
718   public void batch(List<? extends Row> actions, Object[] results) throws IOException {
719     throw new IOException("batch not supported");
720   }
721 
722   @Override
723   public Object[] batch(List<? extends Row> actions) throws IOException {
724     throw new IOException("batch not supported");
725   }
726 
727   @Override
728   public <R> void batchCallback(List<? extends Row> actions, Object[] results,
729       Batch.Callback<R> callback) throws IOException, InterruptedException {
730     throw new IOException("batchCallback not supported");
731   }
732 
733   @Override
734   public <R> Object[] batchCallback(List<? extends Row> actions, Batch.Callback<R> callback)
735    throws IOException, InterruptedException {
736     throw new IOException("batchCallback not supported");
737   }
738 
739   @Override
740   public CoprocessorRpcChannel coprocessorService(byte[] row) {
741     throw new UnsupportedOperationException("coprocessorService not implemented");
742   }
743 
744   @Override
745   public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
746       byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
747       throws ServiceException, Throwable {
748     throw new UnsupportedOperationException("coprocessorService not implemented");
749   }
750 
751   @Override
752   public <T extends Service, R> void coprocessorService(Class<T> service,
753       byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
754       throws ServiceException, Throwable {
755     throw new UnsupportedOperationException("coprocessorService not implemented");
756   }
757 
758   @Override
759   public void mutateRow(RowMutations rm) throws IOException {
760     throw new IOException("atomicMutation not supported");
761   }
762 
763   @Override
764   public void setAutoFlush(boolean autoFlush) {
765     throw new UnsupportedOperationException("setAutoFlush not implemented");
766   }
767 
768   @Override
769   public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
770     throw new UnsupportedOperationException("setAutoFlush not implemented");
771   }
772 
773   @Override
774   public long getWriteBufferSize() {
775     throw new UnsupportedOperationException("getWriteBufferSize not implemented");
776   }
777 
778   @Override
779   public void setWriteBufferSize(long writeBufferSize) throws IOException {
780     throw new IOException("setWriteBufferSize not supported");
781   }
782 }