hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ted Yu <yuzhih...@gmail.com>
Subject Re: Splits are not preserved during table copy using getAdmin().createTable(tableDescriptor);
Date Fri, 03 Oct 2014 18:18:48 GMT
That's true.

Cheers

On Fri, Oct 3, 2014 at 11:16 AM, Serega Sheypak <serega.sheypak@gmail.com>
wrote:

> Ok, I got it. Really, it doesn't mean which key sequence to pass, right?
> The only requirement is to trim first or last key which is zero bytes
> length.
>
> 2014-10-03 22:08 GMT+04:00 Ted Yu <yuzhihong@gmail.com>:
>
> > startKeys contains an empty byte array at the front and endKeys contains
> > empty
> > byte array at the end.
> >
> > You can strip out the empty byte array from startKeys and pass to table
> > creation API.
> >
> > Cheers
> >
> > On Fri, Oct 3, 2014 at 10:57 AM, Serega Sheypak <
> serega.sheypak@gmail.com>
> > wrote:
> >
> > > Do I have to pass an array of startKeys or an array of endKeys?
> > >
> > > 2014-10-03 20:29 GMT+04:00 Serega Sheypak <serega.sheypak@gmail.com>:
> > >
> > > > So easy, thanks :) I've missed that method.
> > > >
> > > > 2014-10-03 20:23 GMT+04:00 Ted Yu <yuzhihong@gmail.com>:
> > > >
> > > >> You can simplify your code by utilizing the following from HTable:
> > > >>
> > > >>   public byte [][] getStartKeys() throws IOException {
> > > >>
> > > >> No need to sort the keys.
> > > >>
> > > >> Cheers
> > > >>
> > > >> On Fri, Oct 3, 2014 at 9:10 AM, Serega Sheypak <
> > > serega.sheypak@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > Thanks, I'm already updated my end-to-end test, waiting for the
> > > result.
> > > >> >
> > > >> > Here is my code snippet, is it ok?
> > > >> >
> > > >> >  @SneakyThrows(IOException.class)
> > > >> >     private byte[][] collectSplits(String tableName){
> > > >> >         HTable table = new HTable(configuration, tableName);
> > > >> >         int splitSize =
> > > >> > table.getRegionLocations().descendingKeySet().size();
> > > >> >
> > > >> >         if(splitSize < 2){
> > > >> >             LOG.info(String.format("There is only one region
for a
> > > >> > table[%s]. No splits to recreate", tableName));
> > > >> >             return new byte[][]{};
> > > >> >         }
> > > >> >         LOG.info(String.format("There are [%s] regions for a
> > > table[%s].
> > > >> > Skip pre-splitting", splitSize, tableName));
> > > >> >
> > > >> >         byte[][] splits = new byte[splitSize][];
> > > >> >       *  Iterator<HRegionInfo> itr =
> > > >> > table.getRegionLocations().descendingKeySet().iterator();*
> > > >> > *        int index = 0;*
> > > >> > *        while (itr.hasNext()){*
> > > >> > *            splits[index] = itr.next().getStartKey();*
> > > >> > *            index++;*
> > > >> > *        }*
> > > >> >
> > > >> >         table.close();
> > > >> >
> > > >> >         Arrays.sort(splits, Bytes.BYTES_COMPARATOR);
> > > >> >         return splits;
> > > >> >     }
> > > >> >
> > > >> >
> > > >> > 2014-10-03 20:02 GMT+04:00 Ted Yu <yuzhihong@gmail.com>:
> > > >> >
> > > >> > > Take a look at this method in HTable:
> > > >> > >
> > > >> > >   public Pair<byte[][],byte[][]> getStartEndKeys()
throws
> > > IOException
> > > >> {
> > > >> > >
> > > >> > > You would see that the first dimension corresponds to the
number
> > of
> > > >> > > regions.
> > > >> > >
> > > >> > >
> > > >> > > Cheers
> > > >> > >
> > > >> > > On Fri, Oct 3, 2014 at 8:33 AM, Serega Sheypak <
> > > >> serega.sheypak@gmail.com
> > > >> > >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Ok, I found them:
> > > >> > > > hTable.getRegionLocations().descendingKeySet()
> > > >> > > > and
> > > >> > > > HRegionInfo has startKey and endKey
> > > >> > > > I have to prepare splits[][].
> > > >> > > > I'm confused a little, how keys should be placed there?
> > splits[][]
> > > >> > shoud
> > > >> > > by
> > > >> > > > and array with length=region count and width = 2?
> > > >> > > >
> > > >> > > >
> > > >> > > > 2014-10-03 17:57 GMT+04:00 Serega Sheypak <
> > > serega.sheypak@gmail.com
> > > >> >:
> > > >> > > >
> > > >> > > > > Hi, here is my code:
> > > >> > > > >
> > > >> > > > > public void dropIfExistsAndCreate(String sourceTableName,
> > String
> > > >> > > > > newTableName) throws IOException {
> > > >> > > > >         LOG.info(String.format("Use [%s] to create
[%s]",
> > > >> > > > sourceTableName,
> > > >> > > > > newTableName));
> > > >> > > > >         HTableDescriptor descriptor =
> > > >> getDescriptor(sourceTableName);
> > > >> > > > >         dropIfExists(newTableName); // doesn't
matter
> > > >> > > > >         createTable(descriptor, newTableName);
> > > >> > > > >     }
> > > >> > > > >
> > > >> > > > >     private void createTable(HTableDescriptor
descriptor,
> > String
> > > >> > > > > newTableName) throws IOException {
> > > >> > > > >         descriptor.setName(Bytes.toBytes(newTableName));
> > > >> > > > >         getAdmin().createTable(descriptor);
> > > >> > > > >         LOG.info(String.format("Table created[%s]",
> > > >> newTableName));
> > > >> > > > >     }
> > > >> > > > >
> > > >> > > > > My source table has 256 regions, newly created
table has no
> > > >> splits.
> > > >> > > > > Is there any possibility to copy splits from source
table?
> > > >> > > > >
> > > >> > > > > I supposed, that HTableDescriptor encasulates
splits, but it
> > was
> > > >> bad
> > > >> > > > > assumption.
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message