提交 249ab9da 编写于 作者: J johnc

Merge

......@@ -553,7 +553,7 @@ public:
static void oops_do_marking_prologue();
static void oops_do_marking_epilogue();
static bool oops_do_marking_is_active() { return _oops_do_mark_nmethods != NULL; }
DEBUG_ONLY(bool test_oops_do_mark() { return _oops_do_mark_link != NULL; })
bool test_oops_do_mark() { return _oops_do_mark_link != NULL; }
// ScopeDesc for an instruction
ScopeDesc* scope_desc_at(address pc);
......
......@@ -42,9 +42,7 @@ typedef GenericTaskQueueSet<CMTaskQueue> CMTaskQueueSet;
class G1CMIsAliveClosure: public BoolObjectClosure {
G1CollectedHeap* _g1;
public:
G1CMIsAliveClosure(G1CollectedHeap* g1) :
_g1(g1)
{}
G1CMIsAliveClosure(G1CollectedHeap* g1) : _g1(g1) { }
void do_object(oop obj) {
ShouldNotCallThis();
......@@ -111,11 +109,6 @@ class CMBitMapRO VALUE_OBJ_CLASS_SPEC {
return offsetToHeapWord(heapWordToOffset(addr) + 1);
}
void mostly_disjoint_range_union(BitMap* from_bitmap,
size_t from_start_index,
HeapWord* to_start_word,
size_t word_num);
// debugging
NOT_PRODUCT(bool covers(ReservedSpace rs) const;)
};
......@@ -258,60 +251,6 @@ class CMMarkStack VALUE_OBJ_CLASS_SPEC {
void oops_do(OopClosure* f);
};
class CMRegionStack VALUE_OBJ_CLASS_SPEC {
MemRegion* _base;
jint _capacity;
jint _index;
jint _oops_do_bound;
bool _overflow;
public:
CMRegionStack();
~CMRegionStack();
void allocate(size_t size);
// This is lock-free; assumes that it will only be called in parallel
// with other "push" operations (no pops).
void push_lock_free(MemRegion mr);
// Lock-free; assumes that it will only be called in parallel
// with other "pop" operations (no pushes).
MemRegion pop_lock_free();
#if 0
// The routines that manipulate the region stack with a lock are
// not currently used. They should be retained, however, as a
// diagnostic aid.
// These two are the implementations that use a lock. They can be
// called concurrently with each other but they should not be called
// concurrently with the lock-free versions (push() / pop()).
void push_with_lock(MemRegion mr);
MemRegion pop_with_lock();
#endif
bool isEmpty() { return _index == 0; }
bool isFull() { return _index == _capacity; }
bool overflow() { return _overflow; }
void clear_overflow() { _overflow = false; }
int size() { return _index; }
// It iterates over the entries in the region stack and it
// invalidates (i.e. assigns MemRegion()) the ones that point to
// regions in the collection set.
bool invalidate_entries_into_cset();
// This gives an upper bound up to which the iteration in
// invalidate_entries_into_cset() will reach. This prevents
// newly-added entries to be unnecessarily scanned.
void set_oops_do_bound() {
_oops_do_bound = _index;
}
void setEmpty() { _index = 0; clear_overflow(); }
};
class ForceOverflowSettings VALUE_OBJ_CLASS_SPEC {
private:
#ifndef PRODUCT
......@@ -408,7 +347,6 @@ class ConcurrentMark : public CHeapObj {
friend class ConcurrentMarkThread;
friend class CMTask;
friend class CMBitMapClosure;
friend class CSetMarkOopClosure;
friend class CMGlobalObjectClosure;
friend class CMRemarkTask;
friend class CMConcurrentMarkingTask;
......@@ -443,7 +381,6 @@ protected:
CMBitMap _markBitMap2;
CMBitMapRO* _prevMarkBitMap; // completed mark bitmap
CMBitMap* _nextMarkBitMap; // under-construction mark bitmap
bool _at_least_one_mark_complete;
BitMap _region_bm;
BitMap _card_bm;
......@@ -457,7 +394,6 @@ protected:
// For gray objects
CMMarkStack _markStack; // Grey objects behind global finger.
CMRegionStack _regionStack; // Grey regions behind global finger.
HeapWord* volatile _finger; // the global finger, region aligned,
// always points to the end of the
// last claimed region
......@@ -502,18 +438,6 @@ protected:
// verbose level
CMVerboseLevel _verbose_level;
// These two fields are used to implement the optimisation that
// avoids pushing objects on the global/region stack if there are
// no collection set regions above the lowest finger.
// This is the lowest finger (among the global and local fingers),
// which is calculated before a new collection set is chosen.
HeapWord* _min_finger;
// If this flag is true, objects/regions that are marked below the
// finger should be pushed on the stack(s). If this is flag is
// false, it is safe not to push them on the stack(s).
bool _should_gray_objects;
// All of these times are in ms.
NumberSeq _init_times;
NumberSeq _remark_times;
......@@ -604,7 +528,7 @@ protected:
CMTaskQueueSet* task_queues() { return _task_queues; }
// Access / manipulation of the overflow flag which is set to
// indicate that the global stack or region stack has overflown
// indicate that the global stack has overflown
bool has_overflown() { return _has_overflown; }
void set_has_overflown() { _has_overflown = true; }
void clear_has_overflown() { _has_overflown = false; }
......@@ -684,68 +608,6 @@ public:
bool mark_stack_overflow() { return _markStack.overflow(); }
bool mark_stack_empty() { return _markStack.isEmpty(); }
// (Lock-free) Manipulation of the region stack
bool region_stack_push_lock_free(MemRegion mr) {
// Currently we only call the lock-free version during evacuation
// pauses.
assert(SafepointSynchronize::is_at_safepoint(), "world should be stopped");
_regionStack.push_lock_free(mr);
if (_regionStack.overflow()) {
set_has_overflown();
return false;
}
return true;
}
// Lock-free version of region-stack pop. Should only be
// called in tandem with other lock-free pops.
MemRegion region_stack_pop_lock_free() {
return _regionStack.pop_lock_free();
}
#if 0
// The routines that manipulate the region stack with a lock are
// not currently used. They should be retained, however, as a
// diagnostic aid.
bool region_stack_push_with_lock(MemRegion mr) {
// Currently we only call the lock-based version during either
// concurrent marking or remark.
assert(!SafepointSynchronize::is_at_safepoint() || !concurrent(),
"if we are at a safepoint it should be the remark safepoint");
_regionStack.push_with_lock(mr);
if (_regionStack.overflow()) {
set_has_overflown();
return false;
}
return true;
}
MemRegion region_stack_pop_with_lock() {
// Currently we only call the lock-based version during either
// concurrent marking or remark.
assert(!SafepointSynchronize::is_at_safepoint() || !concurrent(),
"if we are at a safepoint it should be the remark safepoint");
return _regionStack.pop_with_lock();
}
#endif
int region_stack_size() { return _regionStack.size(); }
bool region_stack_overflow() { return _regionStack.overflow(); }
bool region_stack_empty() { return _regionStack.isEmpty(); }
// Iterate over any regions that were aborted while draining the
// region stack (any such regions are saved in the corresponding
// CMTask) and invalidate (i.e. assign to the empty MemRegion())
// any regions that point into the collection set.
bool invalidate_aborted_regions_in_cset();
// Returns true if there are any aborted memory regions.
bool has_aborted_regions();
CMRootRegions* root_regions() { return &_root_regions; }
bool concurrent_marking_in_progress() {
......@@ -774,10 +636,6 @@ public:
return _task_queues->steal(task_num, hash_seed, obj);
}
// It grays an object by first marking it. Then, if it's behind the
// global finger, it also pushes it on the global stack.
void deal_with_reference(oop obj);
ConcurrentMark(ReservedSpace rs, int max_regions);
~ConcurrentMark();
......@@ -810,22 +668,6 @@ public:
inline void grayRoot(oop obj, size_t word_size,
uint worker_id, HeapRegion* hr = NULL);
// It's used during evacuation pauses to gray a region, if
// necessary, and it's MT-safe. It assumes that the caller has
// marked any objects on that region. If _should_gray_objects is
// true and we're still doing concurrent marking, the region is
// pushed on the region stack, if it is located below the global
// finger, otherwise we do nothing.
void grayRegionIfNecessary(MemRegion mr);
// It's used during evacuation pauses to mark and, if necessary,
// gray a single object and it's MT-safe. It assumes the caller did
// not mark the object. If _should_gray_objects is true and we're
// still doing concurrent marking, the objects is pushed on the
// global stack, if it is located below the global finger, otherwise
// we do nothing.
void markAndGrayObjectIfNecessary(oop p);
// It iterates over the heap and for each object it comes across it
// will dump the contents of its reference fields, as well as
// liveness information for the object and its referents. The dump
......@@ -869,10 +711,6 @@ public:
// Do concurrent phase of marking, to a tentative transitive closure.
void markFromRoots();
// Process all unprocessed SATB buffers. It is called at the
// beginning of an evacuation pause.
void drainAllSATBBuffers();
void checkpointRootsFinal(bool clear_all_soft_refs);
void checkpointRootsFinalWork();
void cleanup();
......@@ -899,10 +737,6 @@ public:
_markStack.note_end_of_gc();
}
// Iterate over the oops in the mark stack and all local queues. It
// also calls invalidate_entries_into_cset() on the region stack.
void oops_do(OopClosure* f);
// Verify that there are no CSet oops on the stacks (taskqueues /
// global mark stack), enqueued SATB buffers, per-thread SATB
// buffers, and fingers (global / per-task). The boolean parameters
......@@ -919,40 +753,6 @@ public:
// unless the force parameter is true.
void update_g1_committed(bool force = false);
void complete_marking_in_collection_set();
// It indicates that a new collection set is being chosen.
void newCSet();
// It registers a collection set heap region with CM. This is used
// to determine whether any heap regions are located above the finger.
void registerCSetRegion(HeapRegion* hr);
// Resets the region fields of any active CMTask whose region fields
// are in the collection set (i.e. the region currently claimed by
// the CMTask will be evacuated and may be used, subsequently, as
// an alloc region). When this happens the region fields in the CMTask
// are stale and, hence, should be cleared causing the worker thread
// to claim a new region.
void reset_active_task_region_fields_in_cset();
// Registers the maximum region-end associated with a set of
// regions with CM. Again this is used to determine whether any
// heap regions are located above the finger.
void register_collection_set_finger(HeapWord* max_finger) {
// max_finger is the highest heap region end of the regions currently
// contained in the collection set. If this value is larger than
// _min_finger then we need to gray objects.
// This routine is like registerCSetRegion but for an entire
// collection of regions.
if (max_finger > _min_finger) {
_should_gray_objects = true;
}
}
// Returns "true" if at least one mark has been completed.
bool at_least_one_mark_complete() { return _at_least_one_mark_complete; }
bool isMarked(oop p) const {
assert(p != NULL && p->is_oop(), "expected an oop");
HeapWord* addr = (HeapWord*)p;
......@@ -1164,23 +964,6 @@ private:
// limit of the region this task is scanning, NULL if we're not scanning one
HeapWord* _region_limit;
// This is used only when we scan regions popped from the region
// stack. It records what the last object on such a region we
// scanned was. It is used to ensure that, if we abort region
// iteration, we do not rescan the first part of the region. This
// should be NULL when we're not scanning a region from the region
// stack.
HeapWord* _region_finger;
// If we abort while scanning a region we record the remaining
// unscanned portion and check this field when marking restarts.
// This avoids having to push on the region stack while other
// marking threads may still be popping regions.
// If we were to push the unscanned portion directly to the
// region stack then we would need to using locking versions
// of the push and pop operations.
MemRegion _aborted_region;
// the number of words this task has scanned
size_t _words_scanned;
// When _words_scanned reaches this limit, the regular clock is
......@@ -1268,8 +1051,6 @@ private:
int _global_transfers_to;
int _global_transfers_from;
int _region_stack_pops;
int _regions_claimed;
int _objs_found_on_bitmap;
......@@ -1347,15 +1128,6 @@ public:
bool has_timed_out() { return _has_timed_out; }
bool claimed() { return _claimed; }
// Support routines for the partially scanned region that may be
// recorded as a result of aborting while draining the CMRegionStack
MemRegion aborted_region() { return _aborted_region; }
void set_aborted_region(MemRegion mr)
{ _aborted_region = mr; }
// Clears any recorded partially scanned region
void clear_aborted_region() { set_aborted_region(MemRegion()); }
void set_cm_oop_closure(G1CMOopClosure* cm_oop_closure);
// It grays the object by marking it and, if necessary, pushing it
......@@ -1385,22 +1157,12 @@ public:
// buffers are available.
void drain_satb_buffers();
// It keeps popping regions from the region stack and processing
// them until the region stack is empty.
void drain_region_stack(BitMapClosure* closure);
// moves the local finger to a new location
inline void move_finger_to(HeapWord* new_finger) {
assert(new_finger >= _finger && new_finger < _region_limit, "invariant");
_finger = new_finger;
}
// moves the region finger to a new location
inline void move_region_finger_to(HeapWord* new_finger) {
assert(new_finger < _cm->finger(), "invariant");
_region_finger = new_finger;
}
CMTask(int task_num, ConcurrentMark *cm,
size_t* marked_bytes, BitMap* card_bm,
CMTaskQueue* task_queue, CMTaskQueueSet* task_queues);
......
......@@ -4355,7 +4355,8 @@ G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1,
_during_initial_mark(_g1->g1_policy()->during_initial_mark_pause()),
_mark_in_progress(_g1->mark_in_progress()) { }
void G1ParCopyHelper::mark_object(oop obj) {
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>::mark_object(oop obj) {
#ifdef ASSERT
HeapRegion* hr = _g1->heap_region_containing(obj);
assert(hr != NULL, "sanity");
......@@ -4366,7 +4367,9 @@ void G1ParCopyHelper::mark_object(oop obj) {
_cm->grayRoot(obj, (size_t) obj->size(), _worker_id);
}
void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>
::mark_forwarded_object(oop from_obj, oop to_obj) {
#ifdef ASSERT
assert(from_obj->is_forwarded(), "from obj should be forwarded");
assert(from_obj->forwardee() == to_obj, "to obj should be the forwardee");
......@@ -4388,7 +4391,9 @@ void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
_cm->grayRoot(to_obj, (size_t) from_obj->size(), _worker_id);
}
oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
oop G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>
::copy_to_survivor_space(oop old) {
size_t word_sz = old->size();
HeapRegion* from_region = _g1->heap_region_containing_raw(old);
// +1 to make the -1 indexes valid...
......@@ -4457,8 +4462,8 @@ oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
} else {
// No point in using the slower heap_region_containing() method,
// given that we know obj is in the heap.
_scanner->set_region(_g1->heap_region_containing_raw(obj));
obj->oop_iterate_backwards(_scanner);
_scanner.set_region(_g1->heap_region_containing_raw(obj));
obj->oop_iterate_backwards(&_scanner);
}
} else {
_par_scan_state->undo_allocation(alloc_purpose, obj_ptr, word_sz);
......@@ -4675,67 +4680,74 @@ public:
double start_time_ms = os::elapsedTime() * 1000.0;
_g1h->g1_policy()->record_gc_worker_start_time(worker_id, start_time_ms);
ResourceMark rm;
HandleMark hm;
{
ResourceMark rm;
HandleMark hm;
ReferenceProcessor* rp = _g1h->ref_processor_stw();
ReferenceProcessor* rp = _g1h->ref_processor_stw();
G1ParScanThreadState pss(_g1h, worker_id);
G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss, rp);
G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, rp);
G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss, rp);
G1ParScanThreadState pss(_g1h, worker_id);
G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss, rp);
G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, rp);
G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss, rp);
pss.set_evac_closure(&scan_evac_cl);
pss.set_evac_failure_closure(&evac_failure_cl);
pss.set_partial_scan_closure(&partial_scan_cl);
pss.set_evac_closure(&scan_evac_cl);
pss.set_evac_failure_closure(&evac_failure_cl);
pss.set_partial_scan_closure(&partial_scan_cl);
G1ParScanExtRootClosure only_scan_root_cl(_g1h, &pss, rp);
G1ParScanPermClosure only_scan_perm_cl(_g1h, &pss, rp);
G1ParScanExtRootClosure only_scan_root_cl(_g1h, &pss, rp);
G1ParScanPermClosure only_scan_perm_cl(_g1h, &pss, rp);
G1ParScanAndMarkExtRootClosure scan_mark_root_cl(_g1h, &pss, rp);
G1ParScanAndMarkPermClosure scan_mark_perm_cl(_g1h, &pss, rp);
G1ParScanAndMarkExtRootClosure scan_mark_root_cl(_g1h, &pss, rp);
G1ParScanAndMarkPermClosure scan_mark_perm_cl(_g1h, &pss, rp);
OopClosure* scan_root_cl = &only_scan_root_cl;
OopsInHeapRegionClosure* scan_perm_cl = &only_scan_perm_cl;
OopClosure* scan_root_cl = &only_scan_root_cl;
OopsInHeapRegionClosure* scan_perm_cl = &only_scan_perm_cl;
if (_g1h->g1_policy()->during_initial_mark_pause()) {
// We also need to mark copied objects.
scan_root_cl = &scan_mark_root_cl;
scan_perm_cl = &scan_mark_perm_cl;
}
if (_g1h->g1_policy()->during_initial_mark_pause()) {
// We also need to mark copied objects.
scan_root_cl = &scan_mark_root_cl;
scan_perm_cl = &scan_mark_perm_cl;
}
G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, &pss);
G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, &pss);
pss.start_strong_roots();
_g1h->g1_process_strong_roots(/* not collecting perm */ false,
SharedHeap::SO_AllClasses,
scan_root_cl,
&push_heap_rs_cl,
scan_perm_cl,
worker_id);
pss.end_strong_roots();
pss.start_strong_roots();
_g1h->g1_process_strong_roots(/* not collecting perm */ false,
SharedHeap::SO_AllClasses,
scan_root_cl,
&push_heap_rs_cl,
scan_perm_cl,
worker_id);
pss.end_strong_roots();
{
double start = os::elapsedTime();
G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);
evac.do_void();
double elapsed_ms = (os::elapsedTime()-start)*1000.0;
double term_ms = pss.term_time()*1000.0;
_g1h->g1_policy()->record_obj_copy_time(worker_id, elapsed_ms-term_ms);
_g1h->g1_policy()->record_termination(worker_id, term_ms, pss.term_attempts());
}
_g1h->g1_policy()->record_thread_age_table(pss.age_table());
_g1h->update_surviving_young_words(pss.surviving_young_words()+1);
{
double start = os::elapsedTime();
G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);
evac.do_void();
double elapsed_ms = (os::elapsedTime()-start)*1000.0;
double term_ms = pss.term_time()*1000.0;
_g1h->g1_policy()->record_obj_copy_time(worker_id, elapsed_ms-term_ms);
_g1h->g1_policy()->record_termination(worker_id, term_ms, pss.term_attempts());
}
_g1h->g1_policy()->record_thread_age_table(pss.age_table());
_g1h->update_surviving_young_words(pss.surviving_young_words()+1);
// Clean up any par-expanded rem sets.
HeapRegionRemSet::par_cleanup();
// Clean up any par-expanded rem sets.
HeapRegionRemSet::par_cleanup();
if (ParallelGCVerbose) {
MutexLocker x(stats_lock());
pss.print_termination_stats(worker_id);
if (ParallelGCVerbose) {
MutexLocker x(stats_lock());
pss.print_termination_stats(worker_id);
}
assert(pss.refs()->is_empty(), "should be empty");
// Close the inner scope so that the ResourceMark and HandleMark
// destructors are executed here and are included as part of the
// "GC Worker Time".
}
assert(pss.refs()->is_empty(), "should be empty");
double end_time_ms = os::elapsedTime() * 1000.0;
_g1h->g1_policy()->record_gc_worker_end_time(worker_id, end_time_ms);
}
......@@ -4743,6 +4755,67 @@ public:
// *** Common G1 Evacuation Stuff
// Closures that support the filtering of CodeBlobs scanned during
// external root scanning.
// Closure applied to reference fields in code blobs (specifically nmethods)
// to determine whether an nmethod contains references that point into
// the collection set. Used as a predicate when walking code roots so
// that only nmethods that point into the collection set are added to the
// 'marked' list.
class G1FilteredCodeBlobToOopClosure : public CodeBlobToOopClosure {
class G1PointsIntoCSOopClosure : public OopClosure {
G1CollectedHeap* _g1;
bool _points_into_cs;
public:
G1PointsIntoCSOopClosure(G1CollectedHeap* g1) :
_g1(g1), _points_into_cs(false) { }
bool points_into_cs() const { return _points_into_cs; }
template <class T>
void do_oop_nv(T* p) {
if (!_points_into_cs) {
T heap_oop = oopDesc::load_heap_oop(p);
if (!oopDesc::is_null(heap_oop) &&
_g1->in_cset_fast_test(oopDesc::decode_heap_oop_not_null(heap_oop))) {
_points_into_cs = true;
}
}
}
virtual void do_oop(oop* p) { do_oop_nv(p); }
virtual void do_oop(narrowOop* p) { do_oop_nv(p); }
};
G1CollectedHeap* _g1;
public:
G1FilteredCodeBlobToOopClosure(G1CollectedHeap* g1, OopClosure* cl) :
CodeBlobToOopClosure(cl, true), _g1(g1) { }
virtual void do_code_blob(CodeBlob* cb) {
nmethod* nm = cb->as_nmethod_or_null();
if (nm != NULL && !(nm->test_oops_do_mark())) {
G1PointsIntoCSOopClosure predicate_cl(_g1);
nm->oops_do(&predicate_cl);
if (predicate_cl.points_into_cs()) {
// At least one of the reference fields or the oop relocations
// in the nmethod points into the collection set. We have to
// 'mark' this nmethod.
// Note: Revisit the following if CodeBlobToOopClosure::do_code_blob()
// or MarkingCodeBlobClosure::do_code_blob() change.
if (!nm->test_set_oops_do_mark()) {
do_newly_marked_nmethod(nm);
}
}
}
}
};
// This method is run in a GC worker.
void
......@@ -4764,7 +4837,7 @@ g1_process_strong_roots(bool collecting_perm_gen,
// Walk the code cache w/o buffering, because StarTask cannot handle
// unaligned oop locations.
CodeBlobToOopClosure eager_scan_code_roots(scan_non_heap_roots, /*do_marking=*/ true);
G1FilteredCodeBlobToOopClosure eager_scan_code_roots(this, scan_non_heap_roots);
process_strong_roots(false, // no scoping; this is parallel code
collecting_perm_gen, so,
......@@ -5378,25 +5451,39 @@ void G1CollectedHeap::evacuate_collection_set() {
rem_set()->prepare_for_younger_refs_iterate(true);
assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
double start_par = os::elapsedTime();
double start_par_time_sec = os::elapsedTime();
double end_par_time_sec;
if (G1CollectedHeap::use_parallel_gc_threads()) {
// The individual threads will set their evac-failure closures.
StrongRootsScope srs(this);
if (ParallelGCVerbose) G1ParScanThreadState::print_termination_stats_hdr();
// These tasks use ShareHeap::_process_strong_tasks
assert(UseDynamicNumberOfGCThreads ||
workers()->active_workers() == workers()->total_workers(),
"If not dynamic should be using all the workers");
workers()->run_task(&g1_par_task);
} else {
{
StrongRootsScope srs(this);
g1_par_task.set_for_termination(n_workers);
g1_par_task.work(0);
if (G1CollectedHeap::use_parallel_gc_threads()) {
// The individual threads will set their evac-failure closures.
if (ParallelGCVerbose) G1ParScanThreadState::print_termination_stats_hdr();
// These tasks use ShareHeap::_process_strong_tasks
assert(UseDynamicNumberOfGCThreads ||
workers()->active_workers() == workers()->total_workers(),
"If not dynamic should be using all the workers");
workers()->run_task(&g1_par_task);
} else {
g1_par_task.set_for_termination(n_workers);
g1_par_task.work(0);
}
end_par_time_sec = os::elapsedTime();
// Closing the inner scope will execute the destructor
// for the StrongRootsScope object. We record the current
// elapsed time before closing the scope so that time
// taken for the SRS destructor is NOT included in the
// reported parallel time.
}
double par_time = (os::elapsedTime() - start_par) * 1000.0;
g1_policy()->record_par_time(par_time);
double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
g1_policy()->record_par_time(par_time_ms);
double code_root_fixup_time_ms =
(os::elapsedTime() - end_par_time_sec) * 1000.0;
g1_policy()->record_code_root_fixup_time(code_root_fixup_time_ms);
set_par_threads(0);
......
......@@ -199,7 +199,8 @@ class G1CollectedHeap : public SharedHeap {
friend class OldGCAllocRegion;
// Closures used in implementation.
friend class G1ParCopyHelper;
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
friend class G1ParCopyClosure;
friend class G1IsAliveClosure;
friend class G1EvacuateFollowersClosure;
friend class G1ParScanThreadState;
......@@ -1676,202 +1677,6 @@ protected:
size_t _max_heap_capacity;
};
#define use_local_bitmaps 1
#define verify_local_bitmaps 0
#define oop_buffer_length 256
#ifndef PRODUCT
class GCLabBitMap;
class GCLabBitMapClosure: public BitMapClosure {
private:
ConcurrentMark* _cm;
GCLabBitMap* _bitmap;
public:
GCLabBitMapClosure(ConcurrentMark* cm,
GCLabBitMap* bitmap) {
_cm = cm;
_bitmap = bitmap;
}
virtual bool do_bit(size_t offset);
};
#endif // !PRODUCT
class GCLabBitMap: public BitMap {
private:
ConcurrentMark* _cm;
int _shifter;
size_t _bitmap_word_covers_words;
// beginning of the heap
HeapWord* _heap_start;
// this is the actual start of the GCLab
HeapWord* _real_start_word;
// this is the actual end of the GCLab
HeapWord* _real_end_word;
// this is the first word, possibly located before the actual start
// of the GCLab, that corresponds to the first bit of the bitmap
HeapWord* _start_word;
// size of a GCLab in words
size_t _gclab_word_size;
static int shifter() {
return MinObjAlignment - 1;
}
// how many heap words does a single bitmap word corresponds to?
static size_t bitmap_word_covers_words() {
return BitsPerWord << shifter();
}
size_t gclab_word_size() const {
return _gclab_word_size;
}
// Calculates actual GCLab size in words
size_t gclab_real_word_size() const {
return bitmap_size_in_bits(pointer_delta(_real_end_word, _start_word))
/ BitsPerWord;
}
static size_t bitmap_size_in_bits(size_t gclab_word_size) {
size_t bits_in_bitmap = gclab_word_size >> shifter();
// We are going to ensure that the beginning of a word in this
// bitmap also corresponds to the beginning of a word in the
// global marking bitmap. To handle the case where a GCLab
// starts from the middle of the bitmap, we need to add enough
// space (i.e. up to a bitmap word) to ensure that we have
// enough bits in the bitmap.
return bits_in_bitmap + BitsPerWord - 1;
}
public:
GCLabBitMap(HeapWord* heap_start, size_t gclab_word_size)
: BitMap(bitmap_size_in_bits(gclab_word_size)),
_cm(G1CollectedHeap::heap()->concurrent_mark()),
_shifter(shifter()),
_bitmap_word_covers_words(bitmap_word_covers_words()),
_heap_start(heap_start),
_gclab_word_size(gclab_word_size),
_real_start_word(NULL),
_real_end_word(NULL),
_start_word(NULL) {
guarantee(false, "GCLabBitMap::GCLabBitmap(): don't call this any more");
}
inline unsigned heapWordToOffset(HeapWord* addr) {
unsigned offset = (unsigned) pointer_delta(addr, _start_word) >> _shifter;
assert(offset < size(), "offset should be within bounds");
return offset;
}
inline HeapWord* offsetToHeapWord(size_t offset) {
HeapWord* addr = _start_word + (offset << _shifter);
assert(_real_start_word <= addr && addr < _real_end_word, "invariant");
return addr;
}
bool fields_well_formed() {
bool ret1 = (_real_start_word == NULL) &&
(_real_end_word == NULL) &&
(_start_word == NULL);
if (ret1)
return true;
bool ret2 = _real_start_word >= _start_word &&
_start_word < _real_end_word &&
(_real_start_word + _gclab_word_size) == _real_end_word &&
(_start_word + _gclab_word_size + _bitmap_word_covers_words)
> _real_end_word;
return ret2;
}
inline bool mark(HeapWord* addr) {
guarantee(use_local_bitmaps, "invariant");
assert(fields_well_formed(), "invariant");
if (addr >= _real_start_word && addr < _real_end_word) {
assert(!isMarked(addr), "should not have already been marked");
// first mark it on the bitmap
at_put(heapWordToOffset(addr), true);
return true;
} else {
return false;
}
}
inline bool isMarked(HeapWord* addr) {
guarantee(use_local_bitmaps, "invariant");
assert(fields_well_formed(), "invariant");
return at(heapWordToOffset(addr));
}
void set_buffer(HeapWord* start) {
guarantee(false, "set_buffer(): don't call this any more");
guarantee(use_local_bitmaps, "invariant");
clear();
assert(start != NULL, "invariant");
_real_start_word = start;
_real_end_word = start + _gclab_word_size;
size_t diff =
pointer_delta(start, _heap_start) % _bitmap_word_covers_words;
_start_word = start - diff;
assert(fields_well_formed(), "invariant");
}
#ifndef PRODUCT
void verify() {
// verify that the marks have been propagated
GCLabBitMapClosure cl(_cm, this);
iterate(&cl);
}
#endif // PRODUCT
void retire() {
guarantee(false, "retire(): don't call this any more");
guarantee(use_local_bitmaps, "invariant");
assert(fields_well_formed(), "invariant");
if (_start_word != NULL) {
CMBitMap* mark_bitmap = _cm->nextMarkBitMap();
// this means that the bitmap was set up for the GCLab
assert(_real_start_word != NULL && _real_end_word != NULL, "invariant");
mark_bitmap->mostly_disjoint_range_union(this,
0, // always start from the start of the bitmap
_start_word,
gclab_real_word_size());
_cm->grayRegionIfNecessary(MemRegion(_real_start_word, _real_end_word));
#ifndef PRODUCT
if (use_local_bitmaps && verify_local_bitmaps)
verify();
#endif // PRODUCT
} else {
assert(_real_start_word == NULL && _real_end_word == NULL, "invariant");
}
}
size_t bitmap_size_in_words() const {
return (bitmap_size_in_bits(gclab_word_size()) + BitsPerWord - 1) / BitsPerWord;
}
};
class G1ParGCAllocBuffer: public ParGCAllocBuffer {
private:
bool _retired;
......
......@@ -140,7 +140,6 @@ G1CollectorPolicy::G1CollectorPolicy() :
_summary(new Summary()),
_cur_clear_ct_time_ms(0.0),
_mark_closure_time_ms(0.0),
_root_region_scan_wait_time_ms(0.0),
_cur_ref_proc_time_ms(0.0),
......@@ -944,9 +943,6 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec,
_cur_aux_times_set[i] = false;
}
// This is initialized to zero here and is set during
// the evacuation pause if marking is in progress.
_cur_satb_drain_time_ms = 0.0;
// This is initialized to zero here and is set during the evacuation
// pause if we actually waited for the root region scanning to finish.
_root_region_scan_wait_time_ms = 0.0;
......@@ -1246,11 +1242,6 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
double other_time_ms = elapsed_ms;
// Subtract the SATB drain time. It's initialized to zero at the
// start of the pause and is updated during the pause if marking
// is in progress.
other_time_ms -= _cur_satb_drain_time_ms;
// Subtract the root region scanning wait time. It's initialized to
// zero at the start of the pause.
other_time_ms -= _root_region_scan_wait_time_ms;
......@@ -1261,15 +1252,13 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
other_time_ms -= known_time;
}
// Now subtract the time taken to fix up roots in generated code
other_time_ms -= _cur_collection_code_root_fixup_time_ms;
// Subtract the time taken to clean the card table from the
// current value of "other time"
other_time_ms -= _cur_clear_ct_time_ms;
// Subtract the time spent completing marking in the collection
// set. Note if marking is not in progress during the pause
// the value of _mark_closure_time_ms will be zero.
other_time_ms -= _mark_closure_time_ms;
// TraceGen0Time and TraceGen1Time summary info updating.
_all_pause_times_ms->add(elapsed_ms);
......@@ -1280,16 +1269,8 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
MainBodySummary* body_summary = _summary->main_body_summary();
assert(body_summary != NULL, "should not be null!");
// This will be non-zero iff marking is currently in progress (i.e.
// _g1->mark_in_progress() == true) and the currrent pause was not
// an initial mark pause. Since the body_summary items are NumberSeqs,
// however, they have to be consistent and updated in lock-step with
// each other. Therefore we unconditionally record the SATB drain
// time - even if it's zero.
body_summary->record_satb_drain_time_ms(_cur_satb_drain_time_ms);
body_summary->record_root_region_scan_wait_time_ms(
_root_region_scan_wait_time_ms);
body_summary->record_ext_root_scan_time_ms(ext_root_scan_time);
body_summary->record_satb_filtering_time_ms(satb_filtering_time);
body_summary->record_update_rs_time_ms(update_rs_time);
......@@ -1305,7 +1286,6 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
body_summary->record_parallel_other_time_ms(parallel_other_time);
}
body_summary->record_mark_closure_time_ms(_mark_closure_time_ms);
body_summary->record_clear_ct_time_ms(_cur_clear_ct_time_ms);
// We exempt parallel collection from this check because Alloc Buffer
......@@ -1401,10 +1381,10 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
print_par_stats(2, "Object Copy", _par_last_obj_copy_times_ms);
print_par_stats(2, "Termination", _par_last_termination_times_ms);
print_par_sizes(3, "Termination Attempts", _par_last_termination_attempts);
print_par_stats(2, "GC Worker End", _par_last_gc_worker_end_times_ms);
for (int i = 0; i < _parallel_gc_threads; i++) {
_par_last_gc_worker_times_ms[i] = _par_last_gc_worker_end_times_ms[i] - _par_last_gc_worker_start_times_ms[i];
_par_last_gc_worker_times_ms[i] = _par_last_gc_worker_end_times_ms[i] -
_par_last_gc_worker_start_times_ms[i];
double worker_known_time = _par_last_ext_root_scan_times_ms[i] +
_par_last_satb_filtering_times_ms[i] +
......@@ -1413,10 +1393,13 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
_par_last_obj_copy_times_ms[i] +
_par_last_termination_times_ms[i];
_par_last_gc_worker_other_times_ms[i] = _cur_collection_par_time_ms - worker_known_time;
_par_last_gc_worker_other_times_ms[i] = _par_last_gc_worker_times_ms[i] -
worker_known_time;
}
print_par_stats(2, "GC Worker", _par_last_gc_worker_times_ms);
print_par_stats(2, "GC Worker Other", _par_last_gc_worker_other_times_ms);
print_par_stats(2, "GC Worker Total", _par_last_gc_worker_times_ms);
print_par_stats(2, "GC Worker End", _par_last_gc_worker_end_times_ms);
} else {
print_stats(1, "Ext Root Scanning", ext_root_scan_time);
if (print_marking_info) {
......@@ -1427,9 +1410,7 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
print_stats(1, "Scan RS", scan_rs_time);
print_stats(1, "Object Copying", obj_copy_time);
}
if (print_marking_info) {
print_stats(1, "Complete CSet Marking", _mark_closure_time_ms);
}
print_stats(1, "Code Root Fixup", _cur_collection_code_root_fixup_time_ms);
print_stats(1, "Clear CT", _cur_clear_ct_time_ms);
#ifndef PRODUCT
print_stats(1, "Cur Clear CC", _cur_clear_cc_time_ms);
......@@ -1577,8 +1558,7 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
}
double all_other_time_ms = pause_time_ms -
(update_rs_time + scan_rs_time + obj_copy_time +
_mark_closure_time_ms + termination_time);
(update_rs_time + scan_rs_time + obj_copy_time + termination_time);
double young_other_time_ms = 0.0;
if (young_cset_region_length() > 0) {
......@@ -1705,41 +1685,6 @@ void G1CollectorPolicy::adjust_concurrent_refinement(double update_rs_time,
dcqs.notify_if_necessary();
}
double
G1CollectorPolicy::
predict_young_collection_elapsed_time_ms(size_t adjustment) {
guarantee( adjustment == 0 || adjustment == 1, "invariant" );
G1CollectedHeap* g1h = G1CollectedHeap::heap();
size_t young_num = g1h->young_list()->length();
if (young_num == 0)
return 0.0;
young_num += adjustment;
size_t pending_cards = predict_pending_cards();
size_t rs_lengths = g1h->young_list()->sampled_rs_lengths() +
predict_rs_length_diff();
size_t card_num;
if (gcs_are_young()) {
card_num = predict_young_card_num(rs_lengths);
} else {
card_num = predict_non_young_card_num(rs_lengths);
}
size_t young_byte_size = young_num * HeapRegion::GrainBytes;
double accum_yg_surv_rate =
_short_lived_surv_rate_group->accum_surv_rate(adjustment);
size_t bytes_to_copy =
(size_t) (accum_yg_surv_rate * (double) HeapRegion::GrainBytes);
return
predict_rs_update_time_ms(pending_cards) +
predict_rs_scan_time_ms(card_num) +
predict_object_copy_time_ms(bytes_to_copy) +
predict_young_other_time_ms(young_num) +
predict_constant_other_time_ms();
}
double
G1CollectorPolicy::predict_base_elapsed_time_ms(size_t pending_cards) {
size_t rs_length = predict_rs_length_diff();
......@@ -1973,7 +1918,6 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
print_summary(1, "Object Copy", body_summary->get_obj_copy_seq());
}
}
print_summary(1, "Mark Closure", body_summary->get_mark_closure_seq());
print_summary(1, "Clear CT", body_summary->get_clear_ct_seq());
print_summary(1, "Other", summary->get_other_seq());
{
......@@ -1982,17 +1926,15 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
if (parallel) {
// parallel
NumberSeq* other_parts[] = {
body_summary->get_satb_drain_seq(),
body_summary->get_root_region_scan_wait_seq(),
body_summary->get_parallel_seq(),
body_summary->get_clear_ct_seq()
};
calc_other_times_ms = NumberSeq(summary->get_total_seq(),
4, other_parts);
3, other_parts);
} else {
// serial
NumberSeq* other_parts[] = {
body_summary->get_satb_drain_seq(),
body_summary->get_root_region_scan_wait_seq(),
body_summary->get_update_rs_seq(),
body_summary->get_ext_root_scan_seq(),
......@@ -2001,7 +1943,7 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
body_summary->get_obj_copy_seq()
};
calc_other_times_ms = NumberSeq(summary->get_total_seq(),
7, other_parts);
6, other_parts);
}
check_other_times(1, summary->get_other_seq(), &calc_other_times_ms);
}
......
......@@ -64,7 +64,6 @@ public:
};
class MainBodySummary: public CHeapObj {
define_num_seq(satb_drain) // optional
define_num_seq(root_region_scan_wait)
define_num_seq(parallel) // parallel only
define_num_seq(ext_root_scan)
......@@ -74,7 +73,6 @@ class MainBodySummary: public CHeapObj {
define_num_seq(obj_copy)
define_num_seq(termination) // parallel only
define_num_seq(parallel_other) // parallel only
define_num_seq(mark_closure)
define_num_seq(clear_ct)
};
......@@ -179,7 +177,9 @@ private:
size_t _cur_collection_pause_used_at_start_bytes;
size_t _cur_collection_pause_used_regions_at_start;
double _cur_collection_par_time_ms;
double _cur_satb_drain_time_ms;
double _cur_collection_code_root_fixup_time_ms;
double _cur_clear_ct_time_ms;
double _cur_ref_proc_time_ms;
double _cur_ref_enq_time_ms;
......@@ -226,8 +226,8 @@ private:
double* _par_last_gc_worker_times_ms;
// Each workers 'other' time i.e. the elapsed time of the parallel
// phase of the pause minus the sum of the individual sub-phase
// times for a given worker thread.
// code executed by a worker minus the sum of the individual sub-phase
// times for that worker thread.
double* _par_last_gc_worker_other_times_ms;
// indicates whether we are in young or mixed GC mode
......@@ -488,7 +488,6 @@ public:
get_new_prediction(_non_young_other_cost_per_region_ms_seq);
}
double predict_young_collection_elapsed_time_ms(size_t adjustment);
double predict_base_elapsed_time_ms(size_t pending_cards);
double predict_base_elapsed_time_ms(size_t pending_cards,
size_t scanned_cards);
......@@ -709,7 +708,6 @@ private:
double _cur_mark_stop_world_time_ms;
double _mark_remark_start_sec;
double _mark_cleanup_start_sec;
double _mark_closure_time_ms;
double _root_region_scan_wait_time_ms;
// Update the young list target length either by setting it to the
......@@ -809,10 +807,6 @@ public:
void record_concurrent_mark_init_end(double
mark_init_elapsed_time_ms);
void record_mark_closure_time(double mark_closure_time_ms) {
_mark_closure_time_ms = mark_closure_time_ms;
}
void record_root_region_scan_wait_time(double time_ms) {
_root_region_scan_wait_time_ms = time_ms;
}
......@@ -846,11 +840,6 @@ public:
_par_last_satb_filtering_times_ms[worker_i] = ms;
}
void record_satb_drain_time(double ms) {
assert(_g1->mark_in_progress(), "shouldn't be here otherwise");
_cur_satb_drain_time_ms = ms;
}
void record_update_rs_time(int thread, double ms) {
_par_last_update_rs_times_ms[thread] = ms;
}
......@@ -897,6 +886,10 @@ public:
_cur_collection_par_time_ms = ms;
}
void record_code_root_fixup_time(double ms) {
_cur_collection_code_root_fixup_time_ms = ms;
}
void record_aux_start_time(int i) {
guarantee(i < _aux_num, "should be within range");
_cur_aux_start_times_ms[i] = os::elapsedTime() * 1000.0;
......
......@@ -118,9 +118,11 @@ public:
virtual void do_oop(narrowOop* p) { do_oop_nv(p); }
};
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
class G1ParCopyClosure : public G1ParClosureSuper {
G1ParScanClosure _scanner;
template <class T> void do_oop_work(T* p);
class G1ParCopyHelper : public G1ParClosureSuper {
G1ParScanClosure *_scanner;
protected:
// Mark the object if it's not already marked. This is used to mark
// objects pointed to by roots that are guaranteed not to move
......@@ -134,23 +136,11 @@ protected:
oop copy_to_survivor_space(oop obj);
public:
G1ParCopyHelper(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state,
G1ParScanClosure *scanner) :
G1ParClosureSuper(g1, par_scan_state), _scanner(scanner) { }
};
template <bool do_gen_barrier, G1Barrier barrier, bool do_mark_object>
class G1ParCopyClosure : public G1ParCopyHelper {
G1ParScanClosure _scanner;
template <class T> void do_oop_work(T* p);
public:
G1ParCopyClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state,
ReferenceProcessor* rp) :
_scanner(g1, par_scan_state, rp),
G1ParCopyHelper(g1, par_scan_state, &_scanner) {
G1ParClosureSuper(g1, par_scan_state) {
assert(_ref_processor == NULL, "sanity");
}
......
......@@ -69,9 +69,6 @@
diagnostic(bool, G1TraceConcRefinement, false, \
"Trace G1 concurrent refinement") \
\
product(intx, G1MarkRegionStackSize, 1024 * 1024, \
"Size of the region stack for concurrent marking.") \
\
product(double, G1ConcMarkStepDurationMillis, 10.0, \
"Target duration of individual concurrent marking steps " \
"in milliseconds.") \
......
......@@ -373,10 +373,9 @@ class HeapRegion: public G1OffsetTableContigSpace {
ScrubRemSetClaimValue = 3,
ParVerifyClaimValue = 4,
RebuildRSClaimValue = 5,
CompleteMarkCSetClaimValue = 6,
ParEvacFailureClaimValue = 7,
AggregateCountClaimValue = 8,
VerifyCountClaimValue = 9
ParEvacFailureClaimValue = 6,
AggregateCountClaimValue = 7,
VerifyCountClaimValue = 8
};
inline HeapWord* par_allocate_no_bot_updates(size_t word_size) {
......
/*
* Copyright (c) 1997, 2010, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 1997, 2012, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
......@@ -179,64 +179,6 @@ void BitMap::clear_large_range(idx_t beg, idx_t end) {
clear_range_within_word(bit_index(end_full_word), end);
}
void BitMap::mostly_disjoint_range_union(BitMap* from_bitmap,
idx_t from_start_index,
idx_t to_start_index,
size_t word_num) {
// Ensure that the parameters are correct.
// These shouldn't be that expensive to check, hence I left them as
// guarantees.
guarantee(from_bitmap->bit_in_word(from_start_index) == 0,
"it should be aligned on a word boundary");
guarantee(bit_in_word(to_start_index) == 0,
"it should be aligned on a word boundary");
guarantee(word_num >= 2, "word_num should be at least 2");
intptr_t* from = (intptr_t*) from_bitmap->word_addr(from_start_index);
intptr_t* to = (intptr_t*) word_addr(to_start_index);
if (*from != 0) {
// if it's 0, then there's no point in doing the CAS
while (true) {
intptr_t old_value = *to;
intptr_t new_value = old_value | *from;
intptr_t res = Atomic::cmpxchg_ptr(new_value, to, old_value);
if (res == old_value) break;
}
}
++from;
++to;
for (size_t i = 0; i < word_num - 2; ++i) {
if (*from != 0) {
// if it's 0, then there's no point in doing the CAS
assert(*to == 0, "nobody else should be writing here");
intptr_t new_value = *from;
*to = new_value;
}
++from;
++to;
}
if (*from != 0) {
// if it's 0, then there's no point in doing the CAS
while (true) {
intptr_t old_value = *to;
intptr_t new_value = old_value | *from;
intptr_t res = Atomic::cmpxchg_ptr(new_value, to, old_value);
if (res == old_value) break;
}
}
// the -1 is because we didn't advance them after the final CAS
assert(from ==
(intptr_t*) from_bitmap->word_addr(from_start_index) + word_num - 1,
"invariant");
assert(to == (intptr_t*) word_addr(to_start_index) + word_num - 1,
"invariant");
}
void BitMap::at_put(idx_t offset, bool value) {
if (value) {
set_bit(offset);
......
/*
* Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 1997, 2012, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
......@@ -192,31 +192,6 @@ class BitMap VALUE_OBJ_CLASS_SPEC {
void par_set_range(idx_t beg, idx_t end, RangeSizeHint hint);
void par_clear_range (idx_t beg, idx_t end, RangeSizeHint hint);
// It performs the union operation between subsets of equal length
// of two bitmaps (the target bitmap of the method and the
// from_bitmap) and stores the result to the target bitmap. The
// from_start_index represents the first bit index of the subrange
// of the from_bitmap. The to_start_index is the equivalent of the
// target bitmap. Both indexes should be word-aligned, i.e. they
// should correspond to the first bit on a bitmap word (it's up to
// the caller to ensure this; the method does check it). The length
// of the subset is specified with word_num and it is in number of
// bitmap words. The caller should ensure that this is at least 2
// (smaller ranges are not support to save extra checks). Again,
// this is checked in the method.
//
// Atomicity concerns: it is assumed that any contention on the
// target bitmap with other threads will happen on the first and
// last words; the ones in between will be "owned" exclusively by
// the calling thread and, in fact, they will already be 0. So, the
// method performs a CAS on the first word, copies the next
// word_num-2 words, and finally performs a CAS on the last word.
void mostly_disjoint_range_union(BitMap* from_bitmap,
idx_t from_start_index,
idx_t to_start_index,
size_t word_num);
// Clearing
void clear_large();
inline void clear();
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册