For files that have a Cloudera copyright (and no other copyright
notice), make changes to follow the ASF source file header policy here:
http://www.apache.org/legal/src-headers.html#headers
Specifically:
1) Remove the Cloudera copyright.
2) Modify NOTICE.txt according to
http://www.apache.org/legal/src-headers.html#notice
to follow that format and add a line for Cloudera.
3) Replace or add the existing ASF license text with the one given
on the website.
Much of this change was automatically generated via:
git grep -li 'Copyright.*Cloudera' > modified_files.txt
cat modified_files.txt | xargs perl -n -i -e 'print unless m#Copyright.*Cloudera#i;'
cat modified_files_txt | xargs fix_apache_license.py [1]
Some manual fixups were performed following those steps, especially when
license text was completely missing from the file.
[1] https://gist.github.com/anonymous/ff71292094362fc5c594 with minor
modification to ORIG_LICENSE to match Impala's license text.
Change-Id: I2e0bd8420945b953e1b806041bea4d72a3943d86
Reviewed-on: http://gerrit.cloudera.org:8080/3779
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
This is a first step towards making our scan node single threaded since we are
moving to an execution model where multi-threading is done at the fragment level.
This patch adds a new synchronous HdfsScanner::GetNext() interface and implements
it for the Parquet scanner. The async execution via HdfsScanner::ProcessSplit()
is still supported and is implemented by repeatedly calling GetNext() for
code sharing purposes.
I did not yet add a single-threaded scan node that uses GetNext().
The new code will be excercised by the existing scan node and tests.
Testing: I ran an exhaustive private build which passed. I also
ran a microbenchmark on a big TPCH lineitem table and there was no
significant difference in scan performance.
Change-Id: Iab50770bac05afcda4d3404fb4f53a0104931eb0
Reviewed-on: http://gerrit.cloudera.org:8080/3801
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Alex Behm <alex.behm@cloudera.com>
This patch refactors hdfs-parquet-scanner.cc into several files.
The new responsibilities of each file/component are roughly as follows:
hdfs-parquet-scanner.h/cc
- Creates column readers and uses them to materializes row batches.
- Evaluates runtime filters and conjuncts, populates row batch queue.
parquet-metadata-utils.h/cc
- Contains utilities for validating Parquet file metadata.
- Parses the schema of a Parquet file into our internal schema
representation.
- Resolves SchemaPaths (e.g. from a table descriptor) against
the internal representation of the Parquet file schema.
parquet-column-readers.h/cc
- Contains the per-column data reading, parsing and value
materialization logic.
Testing: A private core/hdfs run passed.
Change-Id: I4c5fd46f9c1a0ff2a4c30ea5a712fbae17c68f92
Reviewed-on: http://gerrit.cloudera.org:8080/3596
Tested-by: Internal Jenkins
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
The text scanner had some pathological behaviour when reading
significantly past the end of it scan range. E.g. reading a 256mb string
that's split across blocks. ScannerContext defaulted to issuing 1kb
reads, even if the scan node requested significantly more data. E.g. if
the Parquet scanner called ReadBytes(16mb), this was chopped up into
1kb reads, which were reassembled in boundary_buffer_.
Increase the minimum read size in this case to 64kb. Reading that amount
of data should not have any significant overhead even if we only read
a few bytes past the end of the scan range.
ScannerContext implements a saner default algorithm that will work better
if scanners make many small reads: it starts with 64kb reads and doubles
the size of each successive read past the end of the scan range. We
also correct pass the 'read_past_size' into GetNextBuffer(), so that
we always read the right amount of data.
Also save some time by pre-sizing the boundary buffer to the correct
size instead of reallocating it multiple times.
Testing:
Add test case that exercises the code paths for very large strings.
Performance:
The queries in the test case are vastly faster than before. E.g. 0.6s
vs ~60s for the count(*) query.
Change-Id: Id90c5dea44f07dba5dd465cf325fbff28be34137
Reviewed-on: http://gerrit.cloudera.org:8080/3518
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Internal Jenkins
This patch adds error checking to the Avro scanner (both the codegen'd
and interepted paths), including out-of-bounds checks and data
validity checks.
I ran a local benchmark using the following queries:
set num_scanner_threads=1;
select count(i) from default.avro_bigints_big; # file contains only longs
select max(l_orderkey) from biglineitem_avro; # file has tpch.lineitem schema
Both benchmark queries see negligable or no performance impact.
This patch adds a new Avro scanner unit test and an end-to-end test
that queries several corrupted files, as well as updates the zig-zag
varlen int unit test.
Change-Id: I801a11c496a128e02c564c2a9c44baa5a97be132
Reviewed-on: http://gerrit.cloudera.org:8080/3072
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
Previously including runtime-state.h or exec-env.h pulled in a huge
number of headers. By replacing all of those includes with forward
declarations, we can reduce the number of headers included when building
each source file.
This required various changes, including splitting header files, and in
one case extracting the nested DiskIoMgr::RequestContext class so that
the RequestContext can be instantiated without the full DiskIoMgr
header.
The payoff is that touching many header files results in significantly
smaller incremental builds. E.g. changes to bloom-filter.h only require
recompiling a handful of files, instead of 100+.
Build time of individual files should also be slightly quicker, since
they pull in fewer headers.
Change-Id: I3b246ad9c3681d649e7bfc969c7fa885c6242d84
Reviewed-on: http://gerrit.cloudera.org:8080/3108
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Internal Jenkins
The cancelled status is propagated in scanner threads to cause them to
shut down once the limit has been satisified, but depending on the code
path and when abort_on_error=false, this internal status would sometimes
incorrectly end up in the error log. Fix this by factoring out the
abort_on_error handling code so that it's handled more consistently
across scanners. Parquet, RC, and Avro all suffered from this bug.
Testing: exhastive
Change-Id: I4a91a22608e346ca21a23ea66c855eae54bbced6
Reviewed-on: http://gerrit.cloudera.org:8080/2964
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
This change replaces all instances of MemPool::Allocate() in
avro, text, hbase scanners with MemPool::TryAllocate().
HdfsAvroScanner::MaterializeTuple() has been converted to return
a boolean in case of memory allocation failure. The codegen'ed
version of MaterializeTuple() will also return a boolean. In the
future, we should consider returning Status directly but that will
be more involved and best left as a separate change.
Change-Id: I3e5a56501967a58513888917db5ce66dec4fb5ce
Reviewed-on: http://gerrit.cloudera.org:8080/2568
Reviewed-by: Michael Ho <kwho@cloudera.com>
Tested-by: Internal Jenkins
This change replaces all calls to MemPool::Allocate() with
MemPool::TryAllocate() in the parquet scanner and the decompressor.
Also streamline CheckQueryState() to avoid unnecessary spinlock
acquisition for the common case when there is no error. Also
removes some dead code in the text converter.
MemPool::Allocate() is also updated to return a valid pointer
instead of NULL when the allocation size is zero. NULL is only
returned during allocation failure.
This change also updates CollectionValueBuilder::GetFreeMemory()
to return Status in case it exceeds memory limit. As part of the
change, the max allocation limit (2 GB) is also removed from it
as 64-bit allocations are supported in MemPool with this change.
Change-Id: Ic70400407b7662999332448f4d1bce2cc344ca89
Reviewed-on: http://gerrit.cloudera.org:8080/2203
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
On OS X size_t is a typedef to long long instead of being a #define to
long, this leads to a series of compile errors that are addressed in
this patch. In addition, on Linux the 1L literal is a long iteral and
int64_t is a long compatible type. However, on OS X, int64_t is a long
long type and thus not compatible. These issues are fixed in this patch.
The same is true for passing size_t values to the rapidjson API which
only supports uint64_t values as a similar type and thus the paramters
need to be casted.
Change-Id: Id25ad04deb5e88289741f08a8bf85fbf9fb438d6
Reviewed-on: http://gerrit.cloudera.org:8080/1607
Reviewed-by: Martin Grund <mgrund@cloudera.com>
Readability: Martin Grund <mgrund@cloudera.com>
Tested-by: Internal Jenkins
By doing so, we avoid unnecessarily calling the copy constructor for
Status OK objects and loading the value from memory (due to the old
Status::OK being a global). The impact of this patch was validated by
inspecting both optimized assembly code and generated IR code.
Applying this patch has some effect on the amount of generated code. The
new tool `get_code_size` will list the text, data, and bss sizes for all
archives that we produce in a release build. This patch reduces the code
size by ~20 kB.
Text Data BSS
Old 10578622 576864 40825
New 10559367 576864 40809
The majority of the changes in this patch have been mechanically applied
using:
find be/src -name "*.cc" -or -name "*.h" | xargs sed -i
's/Status::OK;/Status::OK\(\);/'
A new micro-benchmark was added to determine the overhead of using
Status in hot code sections.
Machine Info: Intel(R) Core(TM) i7-4770 CPU @ 3.40GHz
status: Function Rate (iters/ms) Comparison
----------------------------------------------------------------------
Call Status::OK() 9.555e+08 1X
Call static Status::Error 4.515e+07 0.04725X
Call Status(Code, 'string') 9.873e+06 0.01033X
Call w/ Assignment 5.422e+08 0.5674X
Call Cond Branch OK 5.941e+06 0.006218X
Call Cond Branch ERROR 7.047e+06 0.007375X
Call Cond Branch Bool (false) 1.914e+10 20.03X
Call Cond Branch Bool (true) 1.491e+11 156X
Call Cond Boost Optional (true) 3.935e+09 4.118X
Call Cond Boost Optional (false) 2.147e+10 22.47X
Change-Id: I1be6f4c52e2db8cba35b3938a236913faa321e9e
Reviewed-on: http://gerrit.cloudera.org:8080/351
Reviewed-by: Martin Grund <mgrund@cloudera.com>
Tested-by: Internal Jenkins
HdfsScanNode::ScannerThread() uses num_unqueued_files_ to determine
whether there are still some scan ranges that haven't yet been enqueued.
If all the scan ranges have been enqueued and GetNextRange() returns
NULL, then the ScannerThread exits.
However, some paths have a large window between when the next set of
scan ranges will be enqueued for a file and the decrement of
num_unqueued_files_. This can lead to excessive busy waiting.
Minimize this window by decrementing the count as soon as we know no new
scanner threads will be needed to process the not yet enqueued scan
ranges (for example, while each parquet column will have a synthesized
scan range, the resulting streams will be consumed by the same thread
that handled the footer range). Also fix compressed text which never
decremented the count.
Change-Id: I6d64ca6d90d2bf496ca458e86c39858b9647c22a
Reviewed-on: http://gerrit.cloudera.org:8080/373
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Dan Hecht <dhecht@cloudera.com>
This patch removes all occurrences of "using namespace std" and "using
namespace boost(.*)" from the codebase. However, there are still cases
where namespace directives are used (e.g. for rapidjson, thrift,
gutil). These have to be tackled in subsequent patches.
To reduce the patch size, this patch introduces a new header file called
"names.h" that will include many of our most frequently used symbols iff
the corresponding include was already added. This means, that this
header file will pull in for example map / string / vector etc, only iff
vector was already included. This requires "common/names.h" to be the
last include. After including `names.h` a new block contains a sorted list
of using definitions (this patch does not fix namespace directive
declarations for other than std / boost namespaces.)
Change-Id: Iebe4c054670d655bc355347e381dae90999cfddf
Reviewed-on: http://gerrit.cloudera.org:8080/338
Reviewed-by: Martin Grund <mgrund@cloudera.com>
Tested-by: Internal Jenkins
This patch passes the file modified time from the file descriptor down
to the scan range. The final goal is to perform caching of the file
handle based on the file name and the modified time.
The motivation behind the patch is as follows: The granularity of the
catalog version number is to coarse to be used to invalidate a cache
entry when HDFS file handles are cached. As a consequence, we intend to
use the file path and the mtime as a combined key to identify if a file
needs to be reopened. If the mtime does not match the one of a cached
handle, the handle will be closed and then reopened.
Change-Id: I39af75b38da0bc8cd317f52c4ad951ce1a0d5a0e
Reviewed-on: http://gerrit.cloudera.org:8080/299
Reviewed-by: Martin Grund <mgrund@cloudera.com>
Tested-by: Internal Jenkins
This patch introduces the concept of error codes for errors that are
recorded in Impala and are going to be presented to the client. These
error codes are used to aggregate and group incoming error / warning
messages to reduce the spill on the shell and increase the usefulness of
the messages. By splitting the message string from the implementation,
it becomes possible to edit the string independently of the code and
pave the way for internationalization.
Error messages are defined as a combination of an enum value and a
string. Both are defined in the Error.thrift file that is automatically
generated using the script in common/thrift/generate_error_codes.py. The
goal of the script is to have a central understandable repository of
error messages. Adding new messages to this file will require rebuilding
the thrift part. The proxy class ErrorMessage is responsible to
represent an error and capture the parameters that are used to format
the error message string.
When error messages are recorded they are recorded based on the
following algorithm:
- If an error message is of type GENERAL, do not aggregate this message
and simply add it to the total number of messages
- If an error messages is of specific type, record the first error
message as a sample and for all other occurrences increment the count.
- The coordinator will merge all error messages except the ones of type
GENERAL and display a count.
For example, in the case of the parquet file spanning multiple blocks
the output will look like:
Parquet files should not be split into multiple hdfs-blocks.
file=hdfs://localhost:20500/fid.parq (1 of 321 similar)
All messages are always logged to VLOG. In the coordinator error
messages are merged across all backends to retain readability in the
case of large clusters.
The current version of this patch adds these new error codes to some of
the most important error messages as a reference implementation.
Change-Id: I1f1811631836d2dd6048035ad33f7194fb71d6b8
Reviewed-on: http://gerrit.cloudera.org:8080/39
Reviewed-by: Martin Grund <mgrund@cloudera.com>
Tested-by: Internal Jenkins
This is so that each partition (even within the same table) can live on
different filesystems. Prior to this change, each partition had to be
on the same filesystem as the table's base directory.
In order to avoid lock contention on the process-wide Hdfs FS cache (now
that the FS connection is looked up for each file desc rather than once
per scan node), a local cache is used so that the process-wide lock can
be avoided on subsequent lookups of the same filesystem.
Change-Id: I300d6f23b101da064f2c30c75f5801e121fd7c45
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/5761
Reviewed-by: Daniel Hecht <dhecht@cloudera.com>
Tested-by: Daniel Hecht <dhecht@cloudera.com>
DistributedFileSystem is lenient about seeking past the end of the file.
Other FileSystem implementations, such as NativeS3FileSystem, return an
error on this condition. That leads to a scary looking message in the
query warnings.
So, when creating scan ranges, let's require that the ranges fall within
the file bounds (at least according to what the HdfsFileDesc indicates
is the length). There were a couple of kinds of AllocateScanRange()
callsites that needed to be fixed up:
1) When a stream wants to read past a scan range, be careful not to read
past the end of the file.
2) When Impala needs to "guess" at the length of a range, use the
file_length as an upper bound on the guess. We were already doing this
someplaces but not everywhere.
3) When the scan range is derived from parquet metadata, validate the
metadata against file_length and issue appropriate errors. This will
give better diagnostics for corrupt files.
Note that we can't rely on this for safety (HdfsFileDesc file_length may
be stale), but it does mean that when metadata is up-to-date Impala will
no longer try to access beyond the end of files (and so we'll no longer
get false positive errors from the filesystem).
Additionally, this change revealed a pre-existing problem with files
that have multiple row-groups. The first time through InitColumns(),
stream_ was set to NULL. But, stream_->filename could potentially be
accessed when constructing error statuses for subsequent row-groups.
Change-Id: Ia668fa8c261547f85a18a96422846edcea57043e
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/5424
Reviewed-by: Daniel Hecht <dhecht@cloudera.com>
Tested-by: jenkins
With this patch, we mark every scan range that is expected to be a
local read according to the file block metadata from the FE. After
finishing the scan range, we check if it was actually local and if
not, log a warning and add it to the runtime profile. This will help
us diagnose cases where the metadata is stale, causing remote reads.
Change-Id: I546da4ffb80d0af8ff8f5120e9639f9943f8b77d
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/5115
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: Skye Wanderman-Milne <skye@cloudera.com>
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/5184
This patch changes the interface for evaluating expressions, in order
to allow for thread-safe expression evaluations and easier
codegen. Thread safety is achieved via the ExprContext class, a
light-weight container for expression tree evaluation state. Codegen
is easier because more expressions can be cross-compiled to IR.
See expr.h and expr-context.h for an overview of the API
changes. See sort-exec-exprs.cc for a simple example of the new
interface and hdfs-scanner.cc for a more complicated example.
This patch has not been completely code reviewed and may need further
cleanup/stylistic work, as well as additional perf work.
Change-Id: I3e3baf14ebffd2687533d0cc01a6fb8ac4def849
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/3459
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: Skye Wanderman-Milne <skye@cloudera.com>
The current codec interface does not support large files/buffers as the lengths are
passed as ints (int32_t). This changes either uses int64_t or calls a safe(r) version
of ProcessBlock() that checks for overflows.
Change-Id: I5820a707f1e11e7a617609d0956bb9a89e3c02ad
Adding the ability to read compressed text. Reading the compression type from the
file descriptors. Trying to homogenize a bit more the interface of the scanners.
Removing the LZO_TEXT file format, since it was not actually a file format.
Modifying the tests to load and test also text/{snap,gzip,bzip} databases.
Note that this patch requires some changes to Impala-lzo as well.
Change-Id: Ic0742ba11f106ba545050bdb71795efbff70ef74
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/3549
Reviewed-by: Ippokratis Pandis <ipandis@cloudera.com>
Reviewed-by: Ishaan Joshi <ishaan@cloudera.com>
Tested-by: Ippokratis Pandis <ipandis@cloudera.com>
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/3651
Tested-by: jenkins
The main improvement introduced by this patch is speeding up
compilation time by pruning the module of unused functions before
running the rest of the optimization passes. This eliminates most of
the functions in the module since it includes all cross-compiled
functions. This requires that all codegen'd functions are registered
with AddFunctionToJit(), so we know which functions can't be deleted.
With this change, the compilation time decreased from 131.398ms to
36.579ms on a simple "select *" query.
The rest of the changes are minor additions to LlvmCodegen that will
be used in the expr refactoring.
Change-Id: If08000d3dc3fd4d777f6d1f7a30639badad89d6c
Reviewed-on: http://gerrit.ent.cloudera.com:8080/2378
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: Skye Wanderman-Milne <skye@cloudera.com>
(cherry picked from commit a1bd583fd743ce39b233f2917e13385179a8c217)
Reviewed-on: http://gerrit.ent.cloudera.com:8080/2572
Tested-by: jenkins
Previously compact_data meant two things and was very confusing.
1) do the tuples from the scan-node need to be compact. i.e. does it need
to copy all the referenced string data into a contiguous mem pool.
2) do the tuples from the scan-node reference io buffers/buffer in the
scanner context.
I've reworked the scanners to understand this better differentiate the two and
minimize copies. The new scanners should prefer
1) reallocating new buffers > copying
2) reusing (decompression/staging) buffers whenever possible
This patch allows scanners to be much more memory efficient reducing the minimum
memory needed from ~120mb to < 40mb.
Change-Id: I1e5af073be9497f8b00947769753b46629b29c3c
(cherry picked from commit 60b8a89d867b6fc24b1206d3b2e85619f1caae68)
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1951
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: Nong Li <nong@cloudera.com>
This patch fixes our handling of Avro tables containing unsupported
types (i.e. complex types). In debug builds we would hit a DCHECK in
codegen, and in release builds we would silently fail in
ResolveSchemas. This patch fixes the DCHECK and propagates the error
properly, as well as removes some redundant logging from other
scanners.
Change-Id: I7f5dcd81404cd85ac1cbb950a2b6abd2d7f6161f
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1902
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: jenkins
(cherry picked from commit 2f84131f3f9d12e5caee339dad417d8f3297b47f)
(cherry picked from commit 2f45d3aaabd88c4dbd2ea2e86a9440273add92e8)
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1940
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
We weren't attaching resources to the row batch when starting a new
row group, so it was possible for string data to be overwritten. This
patch removes CloseStreams() and merges its functionality with
AttachCompletedResources() so it's not possible to destroy streams
without transferring the resources first. It also merges and removes
ScannerContext::Close().
Also adds test cases for IMPALA-720.
Change-Id: Ia8f40c7d39d8702716f1d337fe797e2696bd0fcb
We were previously wasting memory by always reading into 8MB IO
buffers, even when the data read was much less than 8MB. With this
patch, the IO manager picks a buffer size closer to the actual amount
being read (we don't use the exact size so we can continue to recycle
buffers). The minimum IO buffer size is determined via the
--min_buffer_size flag, and the max IO buffer size via the --read_size
flag.
This technique also helps with IMPALA-652, since short columns will
not use as much memory as before (we will not use considerably more
memory than the size of the table).
This patch also changes StringBuffer to use a doubling strategy so it
doesn't end up allocating many large unused buffers, and has the
scanner context use the requested length as the sync read size if it's
larger than the size produced by read_past_size_cb(). These changes
help prevent the boundary buffer in the scanner context from
allocating excess memory.
Change-Id: I0efb3b023ddfddb08bca22d5cb5f9511fb4d6c50
Reviewed-on: http://gerrit.ent.cloudera.com:8080/938
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: jenkins
This patch refactors HDFSScanNode to copy and prepare all conjunct
exprs in Prepare(), rather than in the scanner threads. This is
necessary so the UDF exprs get codegen'd. Prepare() also only codegens
the functions for the necessary file formats now, rather than for all
file formats regardless of what's actually be scanned.
Change-Id: Ic3220cbd0cba9a3baa138b1f50ecdc6889ed0cd1
Reviewed-on: http://gerrit.ent.cloudera.com:8080/710
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: Skye Wanderman-Milne <skye@cloudera.com>
Doing it this way makes sure we don't bail early on the Close path
which is rarely the right thing to do. This found a few places where
we were not doing proper cleanup because of this.
Change-Id: Ie663c68398c14589b5cbc1bd980644b0b10fd865
Reviewed-on: http://gerrit.ent.cloudera.com:8080/373
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: Nong Li <nong@cloudera.com>
Changes MemLimit to MemTracker:
- the limit is optional
- it also records a label and an optional parent
- Consume() and Release() also update the ancestors and there's also a new
AnyLimitExceeded(), which also checks the ancestors
- the consumption counter is a HighwaterMarkCounter and can optionally be created
as part of a profile
Each fragment instance now has a MemTracker that is part of a 3-level
hierarchy: process, query, fragment instance.
Change-Id: I5f580f4956fdf07d70bd9a6531032439aaf0fd07
Reviewed-on: http://gerrit.ent.cloudera.com:8080/339
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: Nong Li <nong@cloudera.com>
This patch changes ScannerContext::read_past_buffer_size_ to be a
callback function. BaseSequenceScanner registers a callback that
computes the average block size to avoid issuing many small
synchronous reads.
Change-Id: I8cdfab4ca6d261d0478d6078466d092aaaa45afd
Reviewed-on: http://gerrit.ent.cloudera.com:8080/253
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: Skye Wanderman-Milne <skye@cloudera.com>
Also includes
- Cleanup of HdfsScanNode/IoMgr interaction
- Rename of ScanRangeContext to scanner context
- Removed files that were no longer being used
- new class MemLimit
- new query flag MEM_LIMIT
- implementation of impalad flag mem_limit
Still missing:
- parsing a mem limit spec that contains "M/G", as in: 1.25G
Rdtsc is not accurate, due to changes in cpu frequency. Very often, the time
reported in the profile is even longer than the time reported by the shell.
This patch replaces Rdtcs with CLOCK_MONOTONIC. It is as fast as Rdtsc and
accurate. It is not affected by cpu frequency changes and it is not affected by
user setting the system clock.
Note that the new profile report will always report time, rather than in clock
cycle. Here's the new profile:
Averaged Fragment 1:(68.241ms 0.00%)
completion times: min:69ms max:69ms mean: 69ms stddev:0
execution rates: min:91.60 KB/sec max:91.60 KB/sec mean:91.60 KB/sec
stddev:0.00 /sec
split sizes: min: 6.32 KB, max: 6.32 KB, avg: 6.32 KB, stddev: 0.00
- RowsProduced: 1
CodeGen:
- CodegenTime: 566.104us <--* reporting in microsec instead of
clock cycle
- CompileTime: 33.202ms
- LoadTime: 2.671ms
- ModuleFileSize: 44.61 KB
DataStreamSender:
- BytesSent: 16.00 B
- DataSinkTime: 50.719us
- SerializeBatchTime: 18.365us
- ThriftTransmitTime: 145.945us
AGGREGATION_NODE (id=1):(68.384ms 15.50%)
- BuildBuckets: 1.02K
- BuildTime: 13.734us
- GetResultsTime: 6.650us
- MemoryUsed: 32.01 KB
- RowsReturned: 1
- RowsReturnedRate: 14.00 /sec
HDFS_SCAN_NODE (id=0):(57.808ms 84.71%)
- BytesRead: 6.32 KB
- DelimiterParseTime: 62.370us
- MaterializeTupleTime: 767ns
- MemoryUsed: 0.00
- PerDiskReadThroughput: 9.32 MB/sec
- RowsReturned: 100
- RowsReturnedRate: 1.73 K/sec
- ScanRangesComplete: 4
- ScannerThreadsInvoluntaryContextSwitches: 0
- ScannerThreadsReadTime: 662.431us
- ScannerThreadsSysTime: 0
- ScannerThreadsTotalWallClockTime: 25ms
- ScannerThreadsUserTime: 0
- ScannerThreadsVoluntaryContextSwitches: 4
- TotalReadThroughput: 0.00 /sec