Missed the last line for the # 2

HCatSchema schema = HCatInputFormat.getTableSchema(job.getConfiguration())


On Mon, Jun 23, 2014 at 8:11 AM, Dmitry Vasilenko <dvasi...@gmail.com>
wrote:

> Hi Brian,
>
> 1. To enumerate databases and tables and to get the Hive table schema you
> can use the code I provided earlier.
>
> 2. To get the HCatalog flavor of the table schema you will use something
> like this:
>
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.mapreduce.InputSplit;
> import org.apache.hadoop.mapreduce.Job;
> import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
> import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
> import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
> import org.apache.hive.hcatalog.mapreduce.HCatSplit;
>
> ...................
>
>   Job job = new Job(configuration);
>   job.setJarByClass(XXXX.class); // you class name
> job.setInputFormatClass(HCatInputFormat.class);
> job.setOutputFormatClass(TextOutputFormat.class);
>   HCatInputFormat.setInput(job, "data_base_name", "table_name",
> "partition_filter");
>
>
> 3. To get input splits you will have to do some gymnastics as the team
> IMHO made an unfortunate decision to change the ReaderContext  in non cross
> version compatible way. The new ReaderContext does not provide you with the
> convenient way to get  input splits so you will have to use HCatInputFormat
> directly:
>
>   ReadEntity.Builder builder = ....
>    ReadEntity entity = builder.build();
>   Job job = new Job(configuration);
> HCatInputFormat inputFormat = HCatInputFormat.setInput(job,
> entity.getDbName(), entity.getTableName(), entity.getFilterString());
>   List<InputSplit> splits =
> inputFormat.getSplits(ShimLoader.getHadoopShims().newJobContext(job));
>
>
>
> 4. To get an iterator over records more effort is needed. Here is the code
> that should do the work for the given input split and Hadoop configuration
> (returned from the Hadoop job)
>
>
> Iterator<HCatRecord> read() throws HCatException {
> HCatInputFormat inputFormat = new HCatInputFormat();
> @SuppressWarnings("rawtypes")
> final RecordReader<WritableComparable, HCatRecord> recordReader;
> try {
> TaskAttemptContext context =
> ShimLoader.getHadoopShims().newTaskAttemptContext(this.configuration, null);
> recordReader = inputFormat.createRecordReader(this.inputSplit, context);
> recordReader.initialize(this.inputSplit, context);
> } catch (Exception e) {
> throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e);
> }
> return new Iterator<HCatRecord>() {
> public boolean hasNext() {
> try {
> boolean retVal = recordReader.nextKeyValue();
> if (retVal) {
> return true;
> }
> // if its false, we need to close recordReader.
> recordReader.close();
> return false;
> } catch (Exception e) {
> throw new RuntimeException(e);
> }
> }
>
> @Override
> public HCatRecord next() {
> try {
> return recordReader.getCurrentValue();
> } catch (Exception e) {
> throw new RuntimeException(e);
> }
> }
>
> @Override
> public void remove() {
> throw new UnsupportedOperationException();
> }
> };
> }
>
> Note also that when you create a Hadoop configuration you will have to
> disable client cache:
>
> configuration.set(HCatConstants.HCAT_HIVE_CLIENT_DISABLE_CACHE,
> Boolean.TRUE.toString());
>
> otherwise the client process will not terminate (HIVE 6268).
>
> Hope this helps.
>
>
>
>
>
>
>
>
>
>
>
>
>
> On Sat, Jun 21, 2014 at 9:11 AM, Brian Jeltema <
> brian.jelt...@digitalenvoy.net> wrote:
>
>> I’m also experimenting with version 0.13, and see that it differs from
>> 0.12 significantly.
>> Can you give me a code example for 0.13?
>>
>> Thanks
>> Brian
>>
>> On Jun 13, 2014, at 9:25 AM, Brian Jeltema <
>> brian.jelt...@digitalenvoy.net> wrote:
>>
>> Version 0.12.0.
>>
>> I’d like to obtain the table’s schema, scan a table partition, and use
>> the schema to parse the rows.
>>
>> I can probably figure this out by looking at the HCatalog source. My
>> concern was that
>> the HCatalog packages in the Hive distributions are excluded in the
>> JavaDoc, which implies
>> that the API is not public. Is there a reason for this?
>>
>> Brian
>>
>> On Jun 13, 2014, at 9:10 AM, Dmitry Vasilenko <dvasi...@gmail.com> wrote:
>>
>> You should be able to access this information. The exact API depends on
>> the version of Hive/HCat. As you know earlier HCat API is being deprecated
>> and will be removed in Hive 0.14.0. I can provide you with the code sample
>> if you tell me what you are trying to do and what version of Hive you are
>> using.
>>
>>
>> On Fri, Jun 13, 2014 at 7:33 AM, Brian Jeltema <
>> brian.jelt...@digitalenvoy.net> wrote:
>>
>>> I’m experimenting with HCatalog, and would like to be able to access
>>> tables and their schema
>>> from a Java application (not Hive/Pig/MapReduce). However, the API seems
>>> to be hidden, which
>>> leads leads me to believe that this is not a supported use case. Is
>>> HCatalog use limited to
>>> one of the supported frameworks?
>>>
>>> TIA
>>>
>>> Brian
>>
>>
>>
>>
>>
>

Reply via email to