org.apache.hadoop.hive.serde2.thrift
Class TCTLSeparatedProtocol

java.lang.Object
  extended by org.apache.thrift.protocol.TProtocol
      extended by org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol
All Implemented Interfaces:
ConfigurableTProtocol, SkippableTProtocol, WriteNullsProtocol

public class TCTLSeparatedProtocol
extends org.apache.thrift.protocol.TProtocol
implements ConfigurableTProtocol, WriteNullsProtocol, SkippableTProtocol

An implementation of the Thrift Protocol for ctl separated records. This is not thrift compliant in that it doesn't write out field ids so things cannot actually be versioned.


Nested Class Summary
static class TCTLSeparatedProtocol.Factory
          Factory for JSON protocol objects.
 
Field Summary
protected  int bufferSize
          The size of the internal buffer to use.
static String BufferSizeKey
           
protected  String[] columns
          For a single row, the split on the primary separator.
protected static String defaultMapSeparator
           
protected static String defaultPrimarySeparator
          These are defaults, but for now leaving them like this.
protected static String defaultRowSeparator
           
protected static String defaultSecondarySeparator
           
protected  long elemIndex
          For writes, on what element are we on so we know when to use normal list separator or for a map know when to use the k/v separator.
protected  String[] fields
          For a single column, a split on the secondary separator.
protected  boolean firstField
          Is this the first field we're writing.
protected  boolean firstInnerField
          Is this the first list/map/set field we're writing for the current element.
protected  int index
          An index into what column we're on.
protected  boolean inner
          Are we currently on the top-level columns or parsing a column itself.
protected  int innerIndex
          An index into what field within a column we're on.
protected  org.apache.thrift.transport.TTransport innerTransport
          The transport being wrapped.
protected  boolean isMap
          Are we writing a map and need to worry about k/v separator?
protected  boolean lastPrimitiveWasNullFlag
           
protected  Pattern mapPattern
           
protected  String mapSeparator
           
protected  String nullString
          The string representing nulls in the serialized data.
protected  org.apache.hadoop.io.Text nullText
          The nullString in UTF-8 bytes.
protected  Pattern primaryPattern
           
protected  String primarySeparator
          The separators for this instance.
protected  String quote
          The quote character when supporting quotes with ability to not split across quoted entries.
protected  boolean returnNulls
          For places where the separators are back to back, should we return a null or an empty string since it is ambiguous.
static String ReturnNullsKey
          Strings used to lookup the various configurable paramaters of this protocol.
protected  String rowSeparator
           
protected  Pattern secondaryPattern
           
protected  String secondarySeparator
           
protected  Pattern stripQuotePostfix
           
protected  Pattern stripQuotePrefix
           
protected  Pattern stripSeparatorPrefix
          For quoted fields, strip away the quotes and also need something to strip away the control separator when using complex split method defined here.
protected  org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.SimpleTransportTokenizer transportTokenizer
          The transport stream is tokenized on the row separator.
 
Fields inherited from class org.apache.thrift.protocol.TProtocol
trans_
 
Constructor Summary
TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans)
          The simple constructor which assumes ctl-a, ctl-b and '\n' separators and to return empty strings for empty fields.
TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans, int buffer_size)
           
TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans, String primarySeparator, String secondarySeparator, String mapSeparator, String rowSeparator, boolean returnNulls, int bufferSize)
           
 
Method Summary
protected  String[] complexSplit(String line, Pattern p)
          Split the line based on a complex regex pattern.
protected  String getByteValue(String altValue, String defaultVal)
           
 String getMapSeparator()
           
 String getPrimarySeparator()
          Inspect the separators this instance is configured with.
 String getRowSeparator()
           
 String getSecondarySeparator()
           
 void initialize(org.apache.hadoop.conf.Configuration conf, Properties tbl)
          Initialize the TProtocol.
protected  void internalInitialize()
          Sets the internal separator patterns and creates the internal tokenizer.
 void internalWriteString(String str)
           
 boolean lastPrimitiveWasNull()
          Was the last primitive read really a NULL.
 ByteBuffer readBinary()
           
 boolean readBool()
           
 byte readByte()
           
 double readDouble()
           
 org.apache.thrift.protocol.TField readFieldBegin()
           
 void readFieldEnd()
           
 short readI16()
           
 int readI32()
           
 long readI64()
           
 org.apache.thrift.protocol.TList readListBegin()
           
 void readListEnd()
           
 org.apache.thrift.protocol.TMap readMapBegin()
           
 void readMapEnd()
           
 org.apache.thrift.protocol.TMessage readMessageBegin()
           
 void readMessageEnd()
           
 org.apache.thrift.protocol.TSet readSetBegin()
           
 void readSetEnd()
           
 String readString()
           
 org.apache.thrift.protocol.TStruct readStructBegin()
           
 void readStructEnd()
           
 void skip(byte type)
          Skip past the current field Just increments the field index counter.
 void writeBinary(ByteBuffer bin)
           
 void writeBool(boolean b)
           
 void writeByte(byte b)
           
 void writeDouble(double dub)
           
 void writeFieldBegin(org.apache.thrift.protocol.TField field)
           
 void writeFieldEnd()
           
 void writeFieldStop()
           
 void writeI16(short i16)
           
 void writeI32(int i32)
           
 void writeI64(long i64)
           
 void writeListBegin(org.apache.thrift.protocol.TList list)
           
 void writeListEnd()
           
 void writeMapBegin(org.apache.thrift.protocol.TMap map)
           
 void writeMapEnd()
           
 void writeMessageBegin(org.apache.thrift.protocol.TMessage message)
           
 void writeMessageEnd()
           
 void writeNull()
          Write a null.
 void writeSetBegin(org.apache.thrift.protocol.TSet set)
           
 void writeSetEnd()
           
 void writeString(String str)
           
 void writeStructBegin(org.apache.thrift.protocol.TStruct struct)
           
 void writeStructEnd()
           
 
Methods inherited from class org.apache.thrift.protocol.TProtocol
getTransport, reset
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Field Detail

defaultPrimarySeparator

protected static final String defaultPrimarySeparator
These are defaults, but for now leaving them like this.

See Also:
Constant Field Values

defaultSecondarySeparator

protected static final String defaultSecondarySeparator
See Also:
Constant Field Values

defaultRowSeparator

protected static final String defaultRowSeparator
See Also:
Constant Field Values

defaultMapSeparator

protected static final String defaultMapSeparator
See Also:
Constant Field Values

primarySeparator

protected String primarySeparator
The separators for this instance.


secondarySeparator

protected String secondarySeparator

rowSeparator

protected String rowSeparator

mapSeparator

protected String mapSeparator

primaryPattern

protected Pattern primaryPattern

secondaryPattern

protected Pattern secondaryPattern

mapPattern

protected Pattern mapPattern

quote

protected String quote
The quote character when supporting quotes with ability to not split across quoted entries. Like csv. Note that escaping the quote is not currently supported.


transportTokenizer

protected org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.SimpleTransportTokenizer transportTokenizer
The transport stream is tokenized on the row separator.


columns

protected String[] columns
For a single row, the split on the primary separator.


index

protected int index
An index into what column we're on.


fields

protected String[] fields
For a single column, a split on the secondary separator.


innerIndex

protected int innerIndex
An index into what field within a column we're on.


firstField

protected boolean firstField
Is this the first field we're writing.


firstInnerField

protected boolean firstInnerField
Is this the first list/map/set field we're writing for the current element.


isMap

protected boolean isMap
Are we writing a map and need to worry about k/v separator?


elemIndex

protected long elemIndex
For writes, on what element are we on so we know when to use normal list separator or for a map know when to use the k/v separator.


inner

protected boolean inner
Are we currently on the top-level columns or parsing a column itself.


returnNulls

protected boolean returnNulls
For places where the separators are back to back, should we return a null or an empty string since it is ambiguous. This also applies to extra columns that are read but aren't in the current record.


innerTransport

protected final org.apache.thrift.transport.TTransport innerTransport
The transport being wrapped.


ReturnNullsKey

public static final String ReturnNullsKey
Strings used to lookup the various configurable paramaters of this protocol.

See Also:
Constant Field Values

BufferSizeKey

public static final String BufferSizeKey
See Also:
Constant Field Values

bufferSize

protected int bufferSize
The size of the internal buffer to use.


nullString

protected String nullString
The string representing nulls in the serialized data. e.g., \N as in mysql.


nullText

protected org.apache.hadoop.io.Text nullText
The nullString in UTF-8 bytes.


stripSeparatorPrefix

protected Pattern stripSeparatorPrefix
For quoted fields, strip away the quotes and also need something to strip away the control separator when using complex split method defined here.


stripQuotePrefix

protected Pattern stripQuotePrefix

stripQuotePostfix

protected Pattern stripQuotePostfix

lastPrimitiveWasNullFlag

protected boolean lastPrimitiveWasNullFlag
Constructor Detail

TCTLSeparatedProtocol

public TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans)
The simple constructor which assumes ctl-a, ctl-b and '\n' separators and to return empty strings for empty fields.

Parameters:
trans - - the ttransport to use as input or output

TCTLSeparatedProtocol

public TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans,
                             int buffer_size)

TCTLSeparatedProtocol

public TCTLSeparatedProtocol(org.apache.thrift.transport.TTransport trans,
                             String primarySeparator,
                             String secondarySeparator,
                             String mapSeparator,
                             String rowSeparator,
                             boolean returnNulls,
                             int bufferSize)
Parameters:
trans - - the ttransport to use as input or output
primarySeparator - the separator between columns (aka fields)
secondarySeparator - the separator within a field for things like sets and maps and lists
mapSeparator - - the key/value separator
rowSeparator - - the record separator
returnNulls - - whether to return a null or an empty string for fields that seem empty (ie two primary separators back to back)
Method Detail

getPrimarySeparator

public String getPrimarySeparator()
Inspect the separators this instance is configured with.


getSecondarySeparator

public String getSecondarySeparator()

getRowSeparator

public String getRowSeparator()

getMapSeparator

public String getMapSeparator()

internalInitialize

protected void internalInitialize()
Sets the internal separator patterns and creates the internal tokenizer.


complexSplit

protected String[] complexSplit(String line,
                                Pattern p)
Split the line based on a complex regex pattern.

Parameters:
line - the current row
p - the pattern for matching fields in the row
Returns:
List of Strings - not including the separator in them

getByteValue

protected String getByteValue(String altValue,
                              String defaultVal)

initialize

public void initialize(org.apache.hadoop.conf.Configuration conf,
                       Properties tbl)
                throws org.apache.thrift.TException
Initialize the TProtocol.

Specified by:
initialize in interface ConfigurableTProtocol
Parameters:
conf - System properties
tbl - table properties
Throws:
org.apache.thrift.TException

writeMessageBegin

public void writeMessageBegin(org.apache.thrift.protocol.TMessage message)
                       throws org.apache.thrift.TException
Specified by:
writeMessageBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeMessageEnd

public void writeMessageEnd()
                     throws org.apache.thrift.TException
Specified by:
writeMessageEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeStructBegin

public void writeStructBegin(org.apache.thrift.protocol.TStruct struct)
                      throws org.apache.thrift.TException
Specified by:
writeStructBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeStructEnd

public void writeStructEnd()
                    throws org.apache.thrift.TException
Specified by:
writeStructEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeFieldBegin

public void writeFieldBegin(org.apache.thrift.protocol.TField field)
                     throws org.apache.thrift.TException
Specified by:
writeFieldBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeFieldEnd

public void writeFieldEnd()
                   throws org.apache.thrift.TException
Specified by:
writeFieldEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeFieldStop

public void writeFieldStop()
Specified by:
writeFieldStop in class org.apache.thrift.protocol.TProtocol

writeMapBegin

public void writeMapBegin(org.apache.thrift.protocol.TMap map)
                   throws org.apache.thrift.TException
Specified by:
writeMapBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeMapEnd

public void writeMapEnd()
                 throws org.apache.thrift.TException
Specified by:
writeMapEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeListBegin

public void writeListBegin(org.apache.thrift.protocol.TList list)
                    throws org.apache.thrift.TException
Specified by:
writeListBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeListEnd

public void writeListEnd()
                  throws org.apache.thrift.TException
Specified by:
writeListEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeSetBegin

public void writeSetBegin(org.apache.thrift.protocol.TSet set)
                   throws org.apache.thrift.TException
Specified by:
writeSetBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeSetEnd

public void writeSetEnd()
                 throws org.apache.thrift.TException
Specified by:
writeSetEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeBool

public void writeBool(boolean b)
               throws org.apache.thrift.TException
Specified by:
writeBool in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeByte

public void writeByte(byte b)
               throws org.apache.thrift.TException
Specified by:
writeByte in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeI16

public void writeI16(short i16)
              throws org.apache.thrift.TException
Specified by:
writeI16 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeI32

public void writeI32(int i32)
              throws org.apache.thrift.TException
Specified by:
writeI32 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeI64

public void writeI64(long i64)
              throws org.apache.thrift.TException
Specified by:
writeI64 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeDouble

public void writeDouble(double dub)
                 throws org.apache.thrift.TException
Specified by:
writeDouble in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

internalWriteString

public void internalWriteString(String str)
                         throws org.apache.thrift.TException
Throws:
org.apache.thrift.TException

writeString

public void writeString(String str)
                 throws org.apache.thrift.TException
Specified by:
writeString in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

writeBinary

public void writeBinary(ByteBuffer bin)
                 throws org.apache.thrift.TException
Specified by:
writeBinary in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readMessageBegin

public org.apache.thrift.protocol.TMessage readMessageBegin()
                                                     throws org.apache.thrift.TException
Specified by:
readMessageBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readMessageEnd

public void readMessageEnd()
                    throws org.apache.thrift.TException
Specified by:
readMessageEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readStructBegin

public org.apache.thrift.protocol.TStruct readStructBegin()
                                                   throws org.apache.thrift.TException
Specified by:
readStructBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readStructEnd

public void readStructEnd()
                   throws org.apache.thrift.TException
Specified by:
readStructEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

skip

public void skip(byte type)
Skip past the current field Just increments the field index counter.

Specified by:
skip in interface SkippableTProtocol

readFieldBegin

public org.apache.thrift.protocol.TField readFieldBegin()
                                                 throws org.apache.thrift.TException
Specified by:
readFieldBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readFieldEnd

public void readFieldEnd()
                  throws org.apache.thrift.TException
Specified by:
readFieldEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readMapBegin

public org.apache.thrift.protocol.TMap readMapBegin()
                                             throws org.apache.thrift.TException
Specified by:
readMapBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readMapEnd

public void readMapEnd()
                throws org.apache.thrift.TException
Specified by:
readMapEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readListBegin

public org.apache.thrift.protocol.TList readListBegin()
                                               throws org.apache.thrift.TException
Specified by:
readListBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readListEnd

public void readListEnd()
                 throws org.apache.thrift.TException
Specified by:
readListEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readSetBegin

public org.apache.thrift.protocol.TSet readSetBegin()
                                             throws org.apache.thrift.TException
Specified by:
readSetBegin in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

lastPrimitiveWasNull

public boolean lastPrimitiveWasNull()
                             throws org.apache.thrift.TException
Description copied from interface: WriteNullsProtocol
Was the last primitive read really a NULL. Need only be called when the value of the primitive was 0. ie the protocol should return 0 on nulls and the caller will then check if it was actually null For boolean this is false.

Specified by:
lastPrimitiveWasNull in interface WriteNullsProtocol
Throws:
org.apache.thrift.TException

writeNull

public void writeNull()
               throws org.apache.thrift.TException
Description copied from interface: WriteNullsProtocol
Write a null.

Specified by:
writeNull in interface WriteNullsProtocol
Throws:
org.apache.thrift.TException

readSetEnd

public void readSetEnd()
                throws org.apache.thrift.TException
Specified by:
readSetEnd in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readBool

public boolean readBool()
                 throws org.apache.thrift.TException
Specified by:
readBool in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readByte

public byte readByte()
              throws org.apache.thrift.TException
Specified by:
readByte in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readI16

public short readI16()
              throws org.apache.thrift.TException
Specified by:
readI16 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readI32

public int readI32()
            throws org.apache.thrift.TException
Specified by:
readI32 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readI64

public long readI64()
             throws org.apache.thrift.TException
Specified by:
readI64 in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readDouble

public double readDouble()
                  throws org.apache.thrift.TException
Specified by:
readDouble in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readString

public String readString()
                  throws org.apache.thrift.TException
Specified by:
readString in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException

readBinary

public ByteBuffer readBinary()
                      throws org.apache.thrift.TException
Specified by:
readBinary in class org.apache.thrift.protocol.TProtocol
Throws:
org.apache.thrift.TException


Copyright © 2013 The Apache Software Foundation