The bug:
Evaluating !empty() predicates at non-scan nodes interacts
poorly with our BE projection of collection slots. For example,
rows could incorrectly be filtered if a !empty() predicate is
assigned to a plan node that comes after the unnest of the
collection that also performs the projection.
The fix:
This patch reworks the generation of !empty() predicates
introduced in IMPALA-2663 for correctness purposes.
The predicates are generated in cases where we can ensure that
they will be assigned only by the parent scan, and no other
plan node.
The conditions are as follows:
- collection table ref is relative and non-correlated
- collection table ref represents the rhs of an inner/cross/semi join
- collection table ref's parent tuple is not outer joined
Change-Id: Ie975ce139a103285c4e9f93c59ce1f1d2aa71767
Reviewed-on: http://gerrit.cloudera.org:8080/2399
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Reviewed-by: Silvius Rus <srus@cloudera.com>
Tested-by: Internal Jenkins
Add a custom cluster test that tests for delays in registering data
stream receivers. We add a stress option to artificially delay this
registration to ensure that it can be handled correctly.
Change-Id: Id5f5746b6023c301bacfa305c525846cdde822c9
Reviewed-on: http://gerrit.cloudera.org:8080/2306
Tested-by: Internal Jenkins
Reviewed-by: Silvius Rus <srus@cloudera.com>
Scanner threads are started in HdfsScanNode::Open(), before the actual
scan ranges are issued in GetNext(). If there is a delay between these
methods, the scanner threads will spin on DiskIoMgr::GetNextRange()
which returns immediately thinking that all the scan ranges are
finished (rather than not yet issued).
This patch works around the issue by forcing scanner threads to wait
until the initial ranges are issued. That wait is limited to 20ms, after
which the thread will wake up to check if it should yield based on
contention for thread tokens; otherwise it waits again.
Fixing this drops CPU usage to nearly 0 during a large delay between
GetNext() and Open() (e.g. by artificially delaying runtime filter
delivery), where before nearly one CPU per scanner thread was consumed.
Change-Id: I064de1ae037b578c70d65503895a920461af877a
Reviewed-on: http://gerrit.cloudera.org:8080/2379
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Internal Jenkins
Until IMPALA-3108 is resolved, this test is blocking almost all Isilon
testing. It fails deterministically, so let's disable it for now so we can
see if any other tests are failing.
Change-Id: I210179c72731aa0e94a3bf9e8f235648b2d0caaf
Reviewed-on: http://gerrit.cloudera.org:8080/2386
Reviewed-by: Taras Bobrovytsky <tbobrovytsky@cloudera.com>
Tested-by: Internal Jenkins
CloseInternal() did not send an EOS RPC if there was an error in
sending the previous RPC or an error in flushing the current row
batch. It instead just logged the error. Due to this, the receiver
on the RPC channel remained open causing the query to hang in the
cluster.
This fix adds a new function called FlushFinal() to the class
DataSink. All the classes that inherit from DataSink
(DataStreamSender included) implement this function. It currently is
a no-op in the classes other than DataStreamSender; this will be
changed in following patches.
FlushFinal() can return a status and all the function calls that could
fail in DataStreamSender::Close() have been moved to FlushFinal()
leaving Close() as a pure teardown function which cannot fail.
Due to FlushFinal() being able to return an error status, the errors
are sent to the PlanFragmentExecutor which sends these errors in a
report via ReportStatusCb() to the coordinator. The coordinator then
cancels the query if it receives a report with an error in it. This
is how the fix helps in getting rid of the cluster hang.
Furthermore, a retry loop is added to send an RPC in
FragmentExecState::ReportStatusCb() in case the coordinator is
unreachable. If this RPC fails to be sent, there is a chance that the
cluster could hang. This will be addressed in a seperate patch.
Change-Id: Ia45d45dedb21c8b8abab266e140766eddfa363fb
Reviewed-on: http://gerrit.cloudera.org:8080/2205
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Henry Robinson <henry@cloudera.com>
This patch adds an output parameter 'already_unregistered' to
FindRecvrOrWait() to signal to the caller in which of two cases it may
have returned NULL. If 'already_unregistered' is true, the receiver has
already been setup and closed (possibly by cancellation, possibly by
the fragment deliberately closing its inputs in the case of a
limit). This is not an error - cancellation will be signalled to the
sender from the coordinator, and deliberate closure means the
coordinator will tear down the query shortly.
If 'already_unregistered' is set to false by FindRecvrOrWait(), the
DataStreamMgr has never seen the intended receiver. This means the
sender has waited for a full timeout period without the upstream
receiver being established; this signals a likely query setup
problem (as long as datastream_sender_timeout_ms is set sufficiently
large) and so we return an error.
We need to tweak the two timeout parameters here:
* datastream_sender_timeout_ms needs to be large enough to avoid false
negatives for problems during query setup (otherwise queries will
unexpectedly cancel that would otherwise have succeeded, if slowly).
* STREAM_EXPIRATION_TIME_MS needs to be set high enough that a query
will not continue executing for longer than STREAM_EXPIRATION_TIME_MS
after it closes its input (otherwise the sender will get
already_unregistered=false, and cancel). This case will only trigger
when a sender tries to call TransmitData() after the receiver has been
closed for STREAM_EXPIRATION_TIME_MS; this should not happen in
non-error cases as receivers are not closed before consuming their
entire input.
In this patch the former has been set to 2 minutes, and the latter to 5
minutes.
Change-Id: Ib1734992c7199b9dd4b03afca5372022051b6fbd
Reviewed-on: http://gerrit.cloudera.org:8080/2305
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Henry Robinson <henry@cloudera.com>
TBloomFilters have a 'directory' structure that is a list of individual
buckets (buckets are about 64k wide). The total size of the directory
can be 1MB or even much more. That leads to a lot of buckets, and very
inefficient deserialisation as each bucket has to be allocated on the
heap.
Instead, this patch changes the TBloomFilter representation to use one
contiguous string (like the real BloomFilter does, so that it can be
allocated with a single operation (and deserialized with a single copy).
This reduces the amount of kernel time used when deserializing a
TBloomFilter by about 20x, and also speeds up converting a TBloomFilter
to a 'real' BloomFilter by about 20x as well.
Change-Id: I5237e776a197cb2696675dbbe0359e751605ed84
Reviewed-on: http://gerrit.cloudera.org:8080/2359
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Internal Jenkins
We shouldn't initialize a HBase connection if Impala will be run on a
cluster without Hbase. This started to bite us with a change to ZK or
HBase that resulted in periodic log messages when a bad HBase connection
was created. This patch changes the logic to only initialize the
question on demand, e.g. when a query is run against a HBase table.
Change-Id: I405f5cbfc62dd4a4aeb6f4019cac358376478884
Reviewed-on: http://gerrit.cloudera.org:8080/2345
Tested-by: Internal Jenkins
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
The bug: On-clause predicates belonging to an inner join were not always assigned
correctly if they referenced an outer-joined tuple. Specifically, our logic
for detecting whether a predicate can be assigned below an outer join if also
left at the outer-join node was not correct, and so we assigned the predicate
below the join, but did not also leave it at the outer join.
The fix: Assign an inner join On-clause conjunct that references an outer-joined
tuple to the join that the On-clause belongs to.
Change-Id: Iffef7718679d48f866fa90fd3257f182cbb385ae
Reviewed-on: http://gerrit.cloudera.org:8080/2309
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Internal Jenkins
Although Impala now restricts the set of users authorized to access
internal APIs to the configured 'be' principal, there are a small number
of uses for allowing other named users to access the same APIs.
This patch adds a configurable whitelist of authorized users as a
command-line flag. By default, it is set to the HDFS user.
Change-Id: If828e86c99c3c398319953b1d3b33d5e3af200da
Reviewed-on: http://gerrit.cloudera.org:8080/2334
Reviewed-by: Sailesh Mukil <sailesh@cloudera.com>
Reviewed-by: Henry Robinson <henry@cloudera.com>
Reviewed-by: Silvius Rus <srus@cloudera.com>
Tested-by: Internal Jenkins
When ReopenClient() calls CreateClient() and CreateClient() fails, it
sets 'client_key' to NULL. ReopenClient(), then on seeing that
CreateClient() has failed has to restore the old client key back to
the 'client_key' so that it doesn't remain NULL.
However, there was a bug in the logic which made 'old_client_key' and
'client_key' point to the same thing thereby effectively leaving
'client_key' permanently NULLed out (because even old_client_key got
NULLed out). This on use (mostly in DoRpc()), caused a crash.
This patch fixes the logic so that the correct old client key is
restored on a failure to CreateClient().
Change-Id: I16699534d5e48acfc2720f23e420d63af7bd2931
Reviewed-on: http://gerrit.cloudera.org:8080/2338
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Reviewed-by: Henry Robinson <henry@cloudera.com>
Reviewed-by: Silvius Rus <srus@cloudera.com>
Tested-by: Internal Jenkins
Fix a bug in which Impala only reads the first stream
of a multi-stream bz2/gzip file.
Changes the bz2 decoder to read the file in a streaming
fashion rather than reading the entire file into memory
before it can be decompressed.
Change-Id: Icbe617d03a69953f0bf3aa0f7c30d34bc612f9f8
(cherry picked from commit b6d0b4e059329633dc50f1f73ebe35b7ac317a8e)
Reviewed-on: http://gerrit.cloudera.org:8080/2219
Reviewed-by: Juan Yu <jyu@cloudera.com>
Tested-by: Internal Jenkins
The dtor of a RuntimeState object disconnects instance_mem_tracker_
from its parent before calling the dtor of its child exec nodes
(which are kept in the obj_pool of RuntimeState). This ensures
that no destroyed MemTracker will be accessed when the MemTracker's
hierarchy (e.g. process MemTracker) is traversed. The above
only works for those child trackers which are destroyed after
the RuntimeState's dtor is called. This is not necessarily the
case for the DataSink object in PlanFragmentExecutor or the
BufferedBlockMgr in RuntimeState itself.
To avoid accessing destroyed MemTrackers of a DataSink object,
this change explicitly unlinks a DataSink object's MemTracker
from its parent at its Close() function. BufferedBlockMgr already
does the same thing in its dtor.
This change also fixes HdfsSequenceTableWriter::Close() so it will
release the memory it has consumed and free MemPool and the memory in it.
Change-Id: I3aec82150a933dc2b261beff41f5f4f022501bfb
Reviewed-on: http://gerrit.cloudera.org:8080/2314
Reviewed-by: Michael Ho <kwho@cloudera.com>
Tested-by: Internal Jenkins
Local file system builds do not allow running more than one impalad
in parallel. This test relies on that behavior and hence is disabled
for such builds.
Change-Id: I93fe6ae37018885ede4838f5a2ce0bf11148c4e6
Reviewed-on: http://gerrit.cloudera.org:8080/2315
Reviewed-by: Bharath Vissapragada <bharathv@cloudera.com>
Tested-by: Internal Jenkins
Previously, one of the constructors of the class RuntimeState
didn't initialize the field 'codegen_expr_'. This may lead to
unexpected use of codegen during expression evaluation
sent from the frontend. This change fixes the problem by
adding the missing initialization of the field.
Change-Id: Ia7757f6c76ea5e38449049935cbd1169254a071a
Reviewed-on: http://gerrit.cloudera.org:8080/2319
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Internal Jenkins
BETWEEN predicates
This commit fixes an issue where the slow path is employed during static
partition pruning for disjunctive BETWEEN predicates, inroducing
significant latency during planning, especially for tables with large
number of partitions.
Change-Id: I66ef566fa176a859d126d49152921a176a491b0a
Reviewed-on: http://gerrit.cloudera.org:8080/2320
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Internal Jenkins
Some micro-optimizations to improve row-filter performance:
1. Hoist size() calls (a pointer subtraction) out of loop conditions in
case the compiler can't tell that it's constant throughout the loop.
2. Move scanner stats to array of structs rather than a struct of arrays
so that a single set of counters is always in a single cache line.
3. Avoid vector<bool> which is specialised to a compact, but slow, bit set.
Change-Id: I10c2f4a2731a40b5cc061d67d0db72edef90a127
Reviewed-on: http://gerrit.cloudera.org:8080/2293
Tested-by: Internal Jenkins
Reviewed-by: Henry Robinson <henry@cloudera.com>
This patch includes two changes designed to make broadcast filters more efficient:
1. Pick a small random subset of fragments to produce broadcast filters
Limit the number of fragment instances that have to produce a broadcast
filter to 3 (the value of MAX_BROADCAST_FILTER_PRODUCERS). Since the
filter is identical across all producers, there is no benefit to
generating it across all identical hash join nodes.
2. Don't publish filters that have local targets
Prevent fragment instances from publishing filters that have local
targets, instead using the short-circuit path to make them instantly
available to their consumer.
Change-Id: I1d771393f755d2be12ae1e3d60142aa3a30381e1
Reviewed-on: http://gerrit.cloudera.org:8080/2255
Tested-by: Internal Jenkins
Reviewed-by: Henry Robinson <henry@cloudera.com>
Per-row filters currently can not evaluate non-slotref expressions, but
the frontend generates them. In some cases, the parquet scanner does not
correctly ignore those filters.
The best fix is to evaluate row filters once a row has been
materialized. This concentrates filter processing per row (better cache
locality), and ensures that a) all slots are available for filter expr
evaluation and b) there is a materialized Tuple available which means
all exprs can be evaluated.
Thankfully the changes required to do this are very simple - just move
logic from the column reader to HdfsParquetScanner::ReadRow().
Change-Id: I75bd07578b90b5acc3b2f2ac38d4ecb8aa3651ba
Reviewed-on: http://gerrit.cloudera.org:8080/2270
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Henry Robinson <henry@cloudera.com>
Trivial queries (e.g. those with only const exprs or limit
0) currently get assigned a mem resource estimate of '0' (to
indicate unknown), which the scheduler treats conservatively
by reserving 4gb/node. This changes the Planner to handle
these trivial queries differently, assigning them a tiny
mem estimate.
Change-Id: I4913d316cec039dc3fffbaecf28d4caa97e398d1
Reviewed-on: http://gerrit.cloudera.org:8080/2308
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: Internal Jenkins
and target exprs
This commit fixes an issue where an AnalysisError is thrown when a
runtime filter has incompatible source and target exprs. This is
triggered when a runtime filter has multiple candidate target scan nodes
not all of which produce a target expr which is cast-compatible with the
source expr.
Change-Id: I544c8fc66915f684ba24d20de525563638c4039d
Reviewed-on: http://gerrit.cloudera.org:8080/2307
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Internal Jenkins
This commit adds a query option, MAX_NUM_RUNTIME_FILTERS, to restrict
the number of runtime filters generated per query. If more than
MAX_NUM_RUNTIME_FILTERS are generated, the runtime filters are sorted by
the selectivity of the associate source join nodes and the
MAX_NUM_RUNTIME_FILTERS most selective filters are applied. Also with
this commit, non-selective filters are automatically discarded, irrespective
of the value of MAX_NUM_RUNTIME_FILTERS.
Change-Id: Ifd41ef6919a6d2b283a8801861a7179c96ed87c6
Reviewed-on: http://gerrit.cloudera.org:8080/2262
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Internal Jenkins
Previously the code disabled runtime filters if a partition was spilled
while building hash tables but not if a partition was spilled when
appending input rows before attempting to build the hash table. Runtime
filters are not compatible with spilling in the current implementation
so need to be disabled in both cases. Prior to the switch to bloom
filters the window of vulnerability was small, because the filters were
limited in size and disabled for the large inputs that would typically
cause this kind of early spilling. When we increased the size of the
runtime filters, it became more likely that the partition would be
spilled and the runtime filters left enabled.
This fix handles both cases and by moving the logic to disable runtime
filters from BuildHashTable() to Spill(). It also adds a DCHECK in
BuildHashTables() to enforce the invariant.
Change-Id: Ia29b715cf0cdc64fd82d8a619c3f25757dca2bd2
Reviewed-on: http://gerrit.cloudera.org:8080/2302
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
Because of IMPALA-2407, we use Linux's COARSE clockid_t on EC2.
However, this has a resolution between 1 and 10 milliseconds, and so
cannot be trusted to measure lower bounds as low as those in
test_hash_join_timer.py.
This is a temporary workaround.
Change-Id: I1332b1d9aede129ea6c508e40e20960fad9414a8
Reviewed-on: http://gerrit.cloudera.org:8080/2298
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
The bug was that we were not properly cloning the params
of a FunctionCallExpr. In a CTAS we analyze the underlying
query stmt twice, the first time on a clone of the original
stmt. The problem was that the first analysis affected the
second analysis due to an improper clone, leading to missing
slots in a scan because the corresponding SlotRefs were
already analyzed.
Change-Id: I0025c0ee54b2f2cb3ba470b26a9de5aa5a3a3ade
Reviewed-on: http://gerrit.cloudera.org:8080/2291
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Internal Jenkins
We need to skip queries that select from tables wiht nested types is
running with the old aggs and joins. To achieve this, move the failing
test to a separate test and use the skip decorator.
Change-Id: Iaf1351c711b524be66a99084657926909425cbff
Reviewed-on: http://gerrit.cloudera.org:8080/2272
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Internal Jenkins
This fixes a rare race in the test where the I/O error resulting
from an initial Unpin() call happens before the second Unpin() call.
The fix is to expect either an OK or a cancelled status.
Also clean up the test a little bit.
Change-Id: Ie2f59ae2e206e9e07574f41f381756f5620d2302
Reviewed-on: http://gerrit.cloudera.org:8080/2278
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Internal Jenkins
type
This commit fixes an issue where in some cases the source and target
exprs of a runtime filter don't have the same type. This was caused by
missing casting information when converting target exprs to using base
table slots.
Change-Id: I0571a3505e2feb24a75106862481633e6e306a37
Reviewed-on: http://gerrit.cloudera.org:8080/2290
Reviewed-by: Henry Robinson <henry@cloudera.com>
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Internal Jenkins
IS [NOT] NULL currently doesn't support complex types. However
that check is missing in the analyzer. This commit adds it.
Change-Id: Ib3bbf27c6b9f1c1e01d91f505bc9b0ca0922adf3
Reviewed-on: http://gerrit.cloudera.org:8080/2289
Reviewed-by: Bharath Vissapragada <bharathv@cloudera.com>
Tested-by: Internal Jenkins
This commit fixes an issue where the DistributedPlanner does not set the
has_local_target flag correctly for the case of a broadcast join when
the target of a runtime filter is on the same plan fragment as the
associated join node.
Change-Id: Id454f94cf967749387ca4b6ff09ab951360e05fa
Reviewed-on: http://gerrit.cloudera.org:8080/2276
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Internal Jenkins
Some TimestampValue converting functions assume caller
ensures TimestampValue instance has a valid date or time
but that's not true. Change those functions to return
result in output parameter and return boolean to indicate
the conversion is good or not.
Change-Id: I7a68a1e14d9c4ee5d83da760d4d76c20c36bc359
(cherry picked from commit 47d8977f5976b9be405f44add966820138fbda6f)
Reviewed-on: http://gerrit.cloudera.org:8080/2195
Reviewed-by: Juan Yu <jyu@cloudera.com>
Tested-by: Internal Jenkins
CLOCK_MONOTONIC_COARSE was added in Linux 2.6.32. Some older operating
systems (e.g. Centos 5) are running on older kernels. Attempting to
reference the CLOCK_MONOTONIC_COARSE preprocessor macro results in
a compile error.
Change-Id: I1520fcb62223384aad1011cae32c706e73ca22c6
Reviewed-on: http://gerrit.cloudera.org:8080/2279
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
By adding a DCHECK at the destructor of MemTracker to verify
that all its consumed memory has been released, various problems
are found in the code. This change fixes all these issues.
The first problem has to do with a memory leak in DataStreamRecvr.
DataStreamRecvr::Close() is responsible for shutting down all the
sender queues and all the row batches in them. However, it tears
down the queues in the wrong order. Specifically, it calls
DataStreamRecvr::SenderQueue::Close() on each sender queue before
unregistering them. There exists a window between the shut down
of the queues and their unregistration which allows other threads
to enqueue into already closed queues. This change fixes the problem
by first unregistering the sender queues before shutting them down.
This guarantees that no new row batch will be created once it has
been unregistered so DataStreamRecvr::SenderQueue::Close() will not
leak any row batches.
The second problem is an accounting problem in which RuntimeFilter
fails to call Release() on query_mem_tracker_ for all the memory it
has consumed. Specifically, if ProcessBuildInput() fails in
ConstructBuildSide() of a PHJ node, the outstanding bloom filters
allocated in AllocatedRuntimeFilters() will not be added to
produced_filters_ set. Since RunFilterBank::Close() only calls
MemTracker::Release() on filters in produced_filters_ and
consumed_filters_ sets, it will miss these 'orphaned' filters.
Fortunately, all allocated filters are kept in the object pool so
there is no memory leak. Since the amount of memory consumed is
already tracked by 'memory_allocated_', this change fixes the
problem by calling MemTracker::Release() with 'memory_allocated_'
in RuntimeFilterbank::Close().
The third problem is that DataStreamSender::Send() frees local
allocations before the expression evaluation instead of after
the evaluation. This leaves some local allocations hanging around
longer than needed. This change moves the location in which local
allocations are freed. In addition, this change also fixes the order
of two fields 'runtime_state_' and 'sink_' in PlanFragmentExecutor.
The wrong order can lead lead to freed memory being accessed in the
destructor of MemTracker and potentially other places.
Change-Id: Ie87fd4cd10a7c7da806ed95eeb262ed51e74ec7b
Reviewed-on: http://gerrit.cloudera.org:8080/2269
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Internal Jenkins
Under concurrency, it was possible for multiple streams to be added to
closed_stream_expirations_ with the same expiration time. This meant
that the clobbered stream would never be removed from
closed_stream_cache_. Fix this by using a multimap instead of a map.
I was able to reproduce this locally and verified that this patch
addresses the problem. The added DCHECK causes the local mini stress
test to fail without using multimap.
Change-Id: I551e3aa9c1c0a0eba9675d27af5d7bff1c6069d2
Reviewed-on: http://gerrit.cloudera.org:8080/2264
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Internal Jenkins
The default value for the pool configuration 'max requests'
is currently an arbitrary 200. We have found that any value
may end up being problematic due to the behavior of some
clients (i.e. Hue and those that behave similarly) which may
leave queries open for a long time, especially in
combination with IMPALA-1575.
In 5.7, we will steer more users towards using admission
control, and the most basic configurations should be the
most permissive to avoid issues for new AC users. In the
future when we can make true memory reservations and better
estimates, we can have default policies that throttle based
on those instead.
Change-Id: Ib66b0e9f185fa483ae2c234a96c33a71b66ac94e
Reviewed-on: http://gerrit.cloudera.org:8080/2259
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
This commit fixes an issue where the drop partition operation in the
catalog (CatalogOpExecutor.alterTableDropPartition()) does not respect
catalog's locking protocol that requires the catalogLock_ to be acquired
before any table-level lock, thereby being susceptible to causing a
deadlock, if executed concurrently with other DDL operations.
Change-Id: I747d0e707546cdf4c38966c3e9e1509efc6b9d06
Reviewed-on: http://gerrit.cloudera.org:8080/2252
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Internal Jenkins
CLOCK_MONOTONIC is 150x slower than CLOCK_MONOTONIC_COARSE on EC2.
This patch changes to CLOCK_MONOTONIC_COARSE unless we are using the
"tsc" clocksource.
Change-Id: I008a9de5d83d712d0b19beea6952c596cab4572b
Reviewed-on: http://gerrit.cloudera.org:8080/2204
Tested-by: Internal Jenkins
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
This patch validates the first component on an internal connection
seeking authorization. If the first component of the requesting user
does not match the first component of the principal validating it,
no authorization is given to that user.
This patch has been manually tested on a secure kerberized cluster.
Change-Id: I57f85ccda7b9004fce8206200eddfbc3c405cb7a
Reviewed-on: http://gerrit.cloudera.org:8080/2237
Reviewed-by: Sailesh Mukil <sailesh@cloudera.com>
Tested-by: Internal Jenkins
If a fragment sends its filters very quickly, Prepare() may not yet have
been called, which means that the RuntimeFilterBank may not have been
initialised. This patch adds coordination between Prepare() and
UpdateFilter() to force the latter to wait for the former.
Change-Id: I76772e8282d93f5952dac225e8020a2868b5478d
Reviewed-on: http://gerrit.cloudera.org:8080/2236
Tested-by: Internal Jenkins
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>