You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Min Zhou (JIRA)" <ji...@apache.org> on 2009/03/13 06:53:50 UTC

[jira] Updated: (HIVE-338) Executing cli commands into thrift server

     [ https://issues.apache.org/jira/browse/HIVE-338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Min Zhou updated HIVE-338:
--------------------------

    Attachment: hiveserver-v1.patch

support add file/jar now


python example

#!/usr/bin/env python

import sys

from hive import ThriftHive
from hive.ttypes import HiveServerException
from thrift import Thrift
from thrift.transport import TSocket
from thrift.transport import TTransport
from thrift.protocol import TBinaryProtocol

try:
    # Make socket
    transport = TSocket.TSocket('localhost', 10000)

    # Buffering is critical. Raw sockets are very slow
    transport = TTransport.TBufferedTransport(transport)

    # Wrap in a protocol
    protocol = TBinaryProtocol.TBinaryProtocol(transport)

    # Create a client to use the protocol encoder
    client = ThriftHive.Client(protocol)

    # Connect!
    transport.open()

    client.execute('ADD FILE /home/zhoumin/py/foo')
    client.execute('ADD FILE /home/zhoumin/py/streaming.py')
    query = '''
        INSERT OVERWRITE TABLE streaming_pokes
        MAP (pokes.foo, pokes.bar)         
            USING 'streaming.py'
        AS new_foo, new_bar
        FROM pokes                                '''

    client.execute(query)
    row = client.fetchOne()
    print row
    # client.execute('select count(mid) from log_data')
    
    # Close!
    transport.close()

except Thrift.TException, tx:
    print '%s' % (tx.message)



java example:

package zhoumin.example;

import java.util.ArrayList;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;

import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.service.HiveClient;
import org.apache.hadoop.hive.service.HiveServerException;

import com.facebook.thrift.TException;
import com.facebook.thrift.protocol.TBinaryProtocol;
import com.facebook.thrift.protocol.TProtocol;
import com.facebook.thrift.transport.TSocket;
import com.facebook.thrift.transport.TTransport;
import com.facebook.thrift.transport.TTransportException;

/**
 * @author jeremy
 * 
 */
public class MyClient {
  public static final int THREADS_NUMBER = 10;
  
  public static class Worker implements Callable<String> {
    
    TTransport transport;
    TProtocol protocol;
    HiveClient client;
    
    public Worker() {
      transport = new TSocket("localhost", 10000);
      protocol = new TBinaryProtocol(transport);
      client = new HiveClient(protocol); 
    }

    public String call() throws Exception {
      transport.open();
      client.execute("add jar /home/zhoumin//hadoop/mapreduce/zhoumin/dist/zhoumin-0.00.1.jar");
      client.execute("CREATE TEMPORARY FUNCTION strlen AS 'taobao.hadoop.hive.udf.UdfStringLength'");
      client.execute("select strlen(mid) from log_data");
      String row = client.fetchOne();
      transport.close();
      return row;
    }
    
  }
  
  public static void main(String[] args) throws TTransportException,
      TException, HiveServerException, MetaException {
    ExecutorService exec = Executors.newCachedThreadPool();
    
    ArrayList<Future<String>> results = new ArrayList<Future<String>>();
    for(int i = 0; i < THREADS_NUMBER; i++) {
      results.add(exec.submit(new Worker()));
    }
    
    for(Future<String> fs : results) {
      try {
        System.out.println(fs.get());
      }catch (InterruptedException e) {
        System.out.println(e);
      } catch(ExecutionException e) {
        System.out.println(e);
      } finally {
        exec.shutdown();
      }
    }
  }
}


the add jar command is also supported on CLI now.

> Executing cli commands into thrift server
> -----------------------------------------
>
>                 Key: HIVE-338
>                 URL: https://issues.apache.org/jira/browse/HIVE-338
>             Project: Hadoop Hive
>          Issue Type: Improvement
>          Components: Server Infrastructure
>    Affects Versions: 0.3.0
>            Reporter: Min Zhou
>         Attachments: hiveserver-v1.patch
>
>
> Let thrift server support set, add/delete file/jar and normal HSQL query.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.