rose: build RoseEngine prototype for bytecode

Rather than tracking a great many offsets in u32 variables, build them
in a RoseEngine prototype.
This commit is contained in:
Justin Viiret 2017-02-23 16:36:28 +11:00 committed by Matthew Barr
parent 90216921b0
commit f4bda9def6
3 changed files with 198 additions and 247 deletions

View File

@ -481,7 +481,7 @@ bool needsMpvCatchup(const RoseBuildImpl &build) {
} }
static static
void fillStateOffsets(const RoseBuildImpl &tbi, u32 rolesWithStateCount, void fillStateOffsets(const RoseBuildImpl &build, u32 rolesWithStateCount,
u32 anchorStateSize, u32 activeArrayCount, u32 anchorStateSize, u32 activeArrayCount,
u32 activeLeftCount, u32 laggedRoseCount, u32 activeLeftCount, u32 laggedRoseCount,
u32 longLitStreamStateRequired, u32 historyRequired, u32 longLitStreamStateRequired, u32 historyRequired,
@ -513,7 +513,7 @@ void fillStateOffsets(const RoseBuildImpl &tbi, u32 rolesWithStateCount,
curr_offset += anchorStateSize; curr_offset += anchorStateSize;
so->groups = curr_offset; so->groups = curr_offset;
so->groups_size = (tbi.group_end + 7) / 8; so->groups_size = (build.group_end + 7) / 8;
assert(so->groups_size <= sizeof(u64a)); assert(so->groups_size <= sizeof(u64a));
curr_offset += so->groups_size; curr_offset += so->groups_size;
@ -523,22 +523,22 @@ void fillStateOffsets(const RoseBuildImpl &tbi, u32 rolesWithStateCount,
// Exhaustion multibit. // Exhaustion multibit.
so->exhausted = curr_offset; so->exhausted = curr_offset;
curr_offset += mmbit_size(tbi.rm.numEkeys()); curr_offset += mmbit_size(build.rm.numEkeys());
// SOM locations and valid/writeable multibit structures. // SOM locations and valid/writeable multibit structures.
if (tbi.ssm.numSomSlots()) { if (build.ssm.numSomSlots()) {
const u32 somWidth = tbi.ssm.somPrecision(); const u32 somWidth = build.ssm.somPrecision();
if (somWidth) { // somWidth is zero in block mode. if (somWidth) { // somWidth is zero in block mode.
curr_offset = ROUNDUP_N(curr_offset, somWidth); curr_offset = ROUNDUP_N(curr_offset, somWidth);
so->somLocation = curr_offset; so->somLocation = curr_offset;
curr_offset += tbi.ssm.numSomSlots() * somWidth; curr_offset += build.ssm.numSomSlots() * somWidth;
} else { } else {
so->somLocation = 0; so->somLocation = 0;
} }
so->somValid = curr_offset; so->somValid = curr_offset;
curr_offset += mmbit_size(tbi.ssm.numSomSlots()); curr_offset += mmbit_size(build.ssm.numSomSlots());
so->somWritable = curr_offset; so->somWritable = curr_offset;
curr_offset += mmbit_size(tbi.ssm.numSomSlots()); curr_offset += mmbit_size(build.ssm.numSomSlots());
} else { } else {
// No SOM handling, avoid growing the stream state any further. // No SOM handling, avoid growing the stream state any further.
so->somLocation = 0; so->somLocation = 0;
@ -2310,17 +2310,16 @@ u32 findMinFloatingLiteralMatch(const RoseBuildImpl &build,
} }
static static
void buildSuffixEkeyLists(const RoseBuildImpl &tbi, build_context &bc, vector<u32> buildSuffixEkeyLists(const RoseBuildImpl &build, build_context &bc,
const QueueIndexFactory &qif, const QueueIndexFactory &qif) {
vector<u32> *out) { vector<u32> out(qif.allocated_count());
out->resize(qif.allocated_count());
map<u32, vector<u32> > qi_to_ekeys; /* for determinism */ map<u32, vector<u32>> qi_to_ekeys; /* for determinism */
for (const auto &e : bc.suffixes) { for (const auto &e : bc.suffixes) {
const suffix_id &s = e.first; const suffix_id &s = e.first;
u32 qi = e.second; u32 qi = e.second;
set<u32> ekeys = reportsToEkeys(all_reports(s), tbi.rm); set<u32> ekeys = reportsToEkeys(all_reports(s), build.rm);
if (!ekeys.empty()) { if (!ekeys.empty()) {
qi_to_ekeys[qi] = {ekeys.begin(), ekeys.end()}; qi_to_ekeys[qi] = {ekeys.begin(), ekeys.end()};
@ -2328,9 +2327,9 @@ void buildSuffixEkeyLists(const RoseBuildImpl &tbi, build_context &bc,
} }
/* for each outfix also build elists */ /* for each outfix also build elists */
for (const auto &outfix : tbi.outfixes) { for (const auto &outfix : build.outfixes) {
u32 qi = outfix.get_queue(); u32 qi = outfix.get_queue();
set<u32> ekeys = reportsToEkeys(all_reports(outfix), tbi.rm); set<u32> ekeys = reportsToEkeys(all_reports(outfix), build.rm);
if (!ekeys.empty()) { if (!ekeys.empty()) {
qi_to_ekeys[qi] = {ekeys.begin(), ekeys.end()}; qi_to_ekeys[qi] = {ekeys.begin(), ekeys.end()};
@ -2338,11 +2337,14 @@ void buildSuffixEkeyLists(const RoseBuildImpl &tbi, build_context &bc,
} }
for (auto &e : qi_to_ekeys) { for (auto &e : qi_to_ekeys) {
assert(!e.second.empty()); u32 qi = e.first;
e.second.push_back(INVALID_EKEY); /* terminator */ auto &ekeys = e.second;
(*out)[e.first] = bc.engine_blob.add(e.second.begin(), assert(!ekeys.empty());
e.second.end()); ekeys.push_back(INVALID_EKEY); /* terminator */
out[qi] = bc.engine_blob.add(ekeys.begin(), ekeys.end());
} }
return out;
} }
/** Returns sparse iter offset in engine blob. */ /** Returns sparse iter offset in engine blob. */
@ -2632,8 +2634,10 @@ u32 writeProgram(build_context &bc, RoseProgram &&program) {
} }
static static
void buildActiveLeftIter(const vector<LeftNfaInfo> &leftTable, vector<mmbit_sparse_iter>
vector<mmbit_sparse_iter> &out) { buildActiveLeftIter(const vector<LeftNfaInfo> &leftTable) {
vector<mmbit_sparse_iter> out;
vector<u32> keys; vector<u32> keys;
for (size_t i = 0; i < leftTable.size(); i++) { for (size_t i = 0; i < leftTable.size(); i++) {
if (!leftTable[i].transient) { if (!leftTable[i].transient) {
@ -2645,11 +2649,11 @@ void buildActiveLeftIter(const vector<LeftNfaInfo> &leftTable,
DEBUG_PRINTF("%zu active roses\n", keys.size()); DEBUG_PRINTF("%zu active roses\n", keys.size());
if (keys.empty()) { if (keys.empty()) {
out.clear(); return out;
return;
} }
mmbBuildSparseIterator(out, keys, leftTable.size()); mmbBuildSparseIterator(out, keys, leftTable.size());
return out;
} }
static static
@ -3376,7 +3380,7 @@ void writeSomOperation(const Report &report, som_operation *op) {
} }
static static
void makeReport(RoseBuildImpl &build, const ReportID id, void makeReport(const RoseBuildImpl &build, const ReportID id,
const bool has_som, RoseProgram &program) { const bool has_som, RoseProgram &program) {
assert(id < build.rm.numReports()); assert(id < build.rm.numReports());
const Report &report = build.rm.getReport(id); const Report &report = build.rm.getReport(id);
@ -3845,7 +3849,7 @@ RoseProgram makeProgram(RoseBuildImpl &build, build_context &bc,
} }
static static
u32 writeBoundaryProgram(RoseBuildImpl &build, build_context &bc, u32 writeBoundaryProgram(const RoseBuildImpl &build, build_context &bc,
const set<ReportID> &reports) { const set<ReportID> &reports) {
if (reports.empty()) { if (reports.empty()) {
return 0; return 0;
@ -3865,13 +3869,10 @@ u32 writeBoundaryProgram(RoseBuildImpl &build, build_context &bc,
} }
static static
RoseBoundaryReports void makeBoundaryPrograms(const RoseBuildImpl &build, build_context &bc,
makeBoundaryPrograms(RoseBuildImpl &build, build_context &bc, const BoundaryReports &boundary,
const BoundaryReports &boundary, const DerivedBoundaryReports &dboundary,
const DerivedBoundaryReports &dboundary) { RoseBoundaryReports &out) {
RoseBoundaryReports out;
memset(&out, 0, sizeof(out));
DEBUG_PRINTF("report ^: %zu\n", boundary.report_at_0.size()); DEBUG_PRINTF("report ^: %zu\n", boundary.report_at_0.size());
DEBUG_PRINTF("report $: %zu\n", boundary.report_at_eod.size()); DEBUG_PRINTF("report $: %zu\n", boundary.report_at_eod.size());
DEBUG_PRINTF("report ^$: %zu\n", dboundary.report_at_0_eod_full.size()); DEBUG_PRINTF("report ^$: %zu\n", dboundary.report_at_0_eod_full.size());
@ -3882,8 +3883,6 @@ makeBoundaryPrograms(RoseBuildImpl &build, build_context &bc,
writeBoundaryProgram(build, bc, boundary.report_at_0); writeBoundaryProgram(build, bc, boundary.report_at_0);
out.reportZeroEodOffset = out.reportZeroEodOffset =
writeBoundaryProgram(build, bc, dboundary.report_at_0_eod_full); writeBoundaryProgram(build, bc, dboundary.report_at_0_eod_full);
return out;
} }
static static
@ -5305,6 +5304,21 @@ size_t calcLongLitThreshold(const RoseBuildImpl &build,
} }
aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) { aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
// We keep all our offsets, counts etc. in a prototype RoseEngine which we
// will copy into the real one once it is allocated: we can't do this
// until we know how big it will be.
RoseEngine proto;
memset(&proto, 0, sizeof(proto));
// Set scanning mode.
if (!cc.streaming) {
proto.mode = HS_MODE_BLOCK;
} else if (cc.vectored) {
proto.mode = HS_MODE_VECTORED;
} else {
proto.mode = HS_MODE_STREAM;
}
DerivedBoundaryReports dboundary(boundary); DerivedBoundaryReports dboundary(boundary);
size_t historyRequired = calcHistoryRequired(); // Updated by HWLM. size_t historyRequired = calcHistoryRequired(); // Updated by HWLM.
@ -5312,11 +5326,11 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
historyRequired); historyRequired);
DEBUG_PRINTF("longLitLengthThreshold=%zu\n", longLitLengthThreshold); DEBUG_PRINTF("longLitLengthThreshold=%zu\n", longLitLengthThreshold);
build_context bc;
groupByFragment(*this); groupByFragment(*this);
auto anchored_dfas = buildAnchoredDfas(*this); auto anchored_dfas = buildAnchoredDfas(*this);
build_context bc;
bc.floatingMinLiteralMatchOffset = bc.floatingMinLiteralMatchOffset =
findMinFloatingLiteralMatch(*this, anchored_dfas); findMinFloatingLiteralMatch(*this, anchored_dfas);
bc.longLitLengthThreshold = longLitLengthThreshold; bc.longLitLengthThreshold = longLitLengthThreshold;
@ -5327,32 +5341,30 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
} }
bc.needs_mpv_catchup = needsMpvCatchup(*this); bc.needs_mpv_catchup = needsMpvCatchup(*this);
auto boundary_out = makeBoundaryPrograms(*this, bc, boundary, dboundary); makeBoundaryPrograms(*this, bc, boundary, dboundary, proto.boundary);
u32 reportProgramOffset; tie(proto.reportProgramOffset, proto.reportProgramCount) =
u32 reportProgramCount;
tie(reportProgramOffset, reportProgramCount) =
buildReportPrograms(*this, bc); buildReportPrograms(*this, bc);
// Build NFAs // Build NFAs
set<u32> no_retrigger_queues;
bool mpv_as_outfix; bool mpv_as_outfix;
prepMpv(*this, bc, &historyRequired, &mpv_as_outfix); prepMpv(*this, bc, &historyRequired, &mpv_as_outfix);
u32 outfixBeginQueue = qif.allocated_count(); proto.outfixBeginQueue = qif.allocated_count();
if (!prepOutfixes(*this, bc, &historyRequired)) { if (!prepOutfixes(*this, bc, &historyRequired)) {
return nullptr; return nullptr;
} }
u32 outfixEndQueue = qif.allocated_count(); proto.outfixEndQueue = qif.allocated_count();
u32 leftfixBeginQueue = outfixEndQueue; proto.leftfixBeginQueue = proto.outfixEndQueue;
set<u32> no_retrigger_queues;
set<u32> eager_queues; set<u32> eager_queues;
/* Note: buildNfas may reduce the lag for vertices that have prefixes */ /* Note: buildNfas may reduce the lag for vertices that have prefixes */
if (!buildNfas(*this, bc, qif, &no_retrigger_queues, &eager_queues, if (!buildNfas(*this, bc, qif, &no_retrigger_queues, &eager_queues,
&leftfixBeginQueue)) { &proto.leftfixBeginQueue)) {
return nullptr; return nullptr;
} }
u32 eodNfaIterOffset = buildEodNfaIterator(bc, leftfixBeginQueue); u32 eodNfaIterOffset = buildEodNfaIterator(bc, proto.leftfixBeginQueue);
buildCountingMiracles(bc); buildCountingMiracles(bc);
u32 queue_count = qif.allocated_count(); /* excludes anchored matcher q; u32 queue_count = qif.allocated_count(); /* excludes anchored matcher q;
@ -5361,15 +5373,14 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
throw ResourceLimitError(); throw ResourceLimitError();
} }
vector<u32> suffixEkeyLists; auto suffixEkeyLists = buildSuffixEkeyLists(*this, bc, qif);
buildSuffixEkeyLists(*this, bc, qif, &suffixEkeyLists);
assignStateIndices(*this, bc); assignStateIndices(*this, bc);
u32 laggedRoseCount = 0; u32 laggedRoseCount = 0;
vector<LeftNfaInfo> leftInfoTable; vector<LeftNfaInfo> leftInfoTable;
buildLeftInfoTable(*this, bc, eager_queues, leftfixBeginQueue, buildLeftInfoTable(*this, bc, eager_queues, proto.leftfixBeginQueue,
queue_count - leftfixBeginQueue, leftInfoTable, queue_count - proto.leftfixBeginQueue, leftInfoTable,
&laggedRoseCount, &historyRequired); &laggedRoseCount, &historyRequired);
// Information only needed for program construction. // Information only needed for program construction.
@ -5377,32 +5388,25 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
prog_build.vertex_group_map = getVertexGroupMap(*this); prog_build.vertex_group_map = getVertexGroupMap(*this);
prog_build.squashable_groups = getSquashableGroups(*this); prog_build.squashable_groups = getSquashableGroups(*this);
u32 anchoredProgramOffset; tie(proto.anchoredProgramOffset, proto.anchored_count) =
u32 anchoredProgramCount;
tie(anchoredProgramOffset, anchoredProgramCount) =
writeAnchoredPrograms(*this, bc, prog_build); writeAnchoredPrograms(*this, bc, prog_build);
u32 delayProgramOffset; tie(proto.delayProgramOffset, proto.delay_count) =
u32 delayProgramCount;
tie(delayProgramOffset, delayProgramCount) =
writeDelayPrograms(*this, bc, prog_build); writeDelayPrograms(*this, bc, prog_build);
buildLiteralPrograms(*this, bc, prog_build); buildLiteralPrograms(*this, bc, prog_build);
u32 eodProgramOffset = proto.eodProgramOffset =
writeEodProgram(*this, bc, prog_build, eodNfaIterOffset); writeEodProgram(*this, bc, prog_build, eodNfaIterOffset);
size_t longLitStreamStateRequired = 0; size_t longLitStreamStateRequired = 0;
u32 longLitTableOffset = buildLongLiteralTable(*this, bc.engine_blob, proto.longLitTableOffset = buildLongLiteralTable(*this, bc.engine_blob,
bc.longLiterals, longLitLengthThreshold, &historyRequired, bc.longLiterals, longLitLengthThreshold, &historyRequired,
&longLitStreamStateRequired); &longLitStreamStateRequired);
vector<mmbit_sparse_iter> activeLeftIter; proto.lastByteHistoryIterOffset = buildLastByteIter(g, bc);
buildActiveLeftIter(leftInfoTable, activeLeftIter); proto.eagerIterOffset = buildEagerQueueIter(
eager_queues, proto.leftfixBeginQueue, queue_count, bc);
u32 lastByteOffset = buildLastByteIter(g, bc);
u32 eagerIterOffset = buildEagerQueueIter(eager_queues, leftfixBeginQueue,
queue_count, bc);
// Enforce role table resource limit. // Enforce role table resource limit.
if (num_vertices(g) > cc.grey.limitRoseRoleCount) { if (num_vertices(g) > cc.grey.limitRoseRoleCount) {
@ -5423,11 +5427,10 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
// Build anchored matcher. // Build anchored matcher.
size_t asize = 0; size_t asize = 0;
u32 amatcherOffset = 0;
auto atable = buildAnchoredMatcher(*this, anchored_dfas, &asize); auto atable = buildAnchoredMatcher(*this, anchored_dfas, &asize);
if (atable) { if (atable) {
currOffset = ROUNDUP_CL(currOffset); currOffset = ROUNDUP_CL(currOffset);
amatcherOffset = currOffset; proto.amatcherOffset = currOffset;
currOffset += verify_u32(asize); currOffset += verify_u32(asize);
} }
@ -5436,10 +5439,9 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
size_t fsize = 0; size_t fsize = 0;
auto ftable = buildFloatingMatcher(*this, bc.longLitLengthThreshold, auto ftable = buildFloatingMatcher(*this, bc.longLitLengthThreshold,
&fgroups, &fsize, &historyRequired); &fgroups, &fsize, &historyRequired);
u32 fmatcherOffset = 0;
if (ftable) { if (ftable) {
currOffset = ROUNDUP_CL(currOffset); currOffset = ROUNDUP_CL(currOffset);
fmatcherOffset = currOffset; proto.fmatcherOffset = currOffset;
currOffset += verify_u32(fsize); currOffset += verify_u32(fsize);
bc.resources.has_floating = true; bc.resources.has_floating = true;
} }
@ -5448,64 +5450,60 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
size_t drsize = 0; size_t drsize = 0;
auto drtable = auto drtable =
buildDelayRebuildMatcher(*this, bc.longLitLengthThreshold, &drsize); buildDelayRebuildMatcher(*this, bc.longLitLengthThreshold, &drsize);
u32 drmatcherOffset = 0;
if (drtable) { if (drtable) {
currOffset = ROUNDUP_CL(currOffset); currOffset = ROUNDUP_CL(currOffset);
drmatcherOffset = currOffset; proto.drmatcherOffset = currOffset;
currOffset += verify_u32(drsize); currOffset += verify_u32(drsize);
} }
// Build EOD-anchored HWLM matcher. // Build EOD-anchored HWLM matcher.
size_t esize = 0; size_t esize = 0;
auto etable = buildEodAnchoredMatcher(*this, &esize); auto etable = buildEodAnchoredMatcher(*this, &esize);
u32 ematcherOffset = 0;
if (etable) { if (etable) {
currOffset = ROUNDUP_CL(currOffset); currOffset = ROUNDUP_CL(currOffset);
ematcherOffset = currOffset; proto.ematcherOffset = currOffset;
currOffset += verify_u32(esize); currOffset += verify_u32(esize);
} }
// Build small-block HWLM matcher. // Build small-block HWLM matcher.
size_t sbsize = 0; size_t sbsize = 0;
auto sbtable = buildSmallBlockMatcher(*this, &sbsize); auto sbtable = buildSmallBlockMatcher(*this, &sbsize);
u32 sbmatcherOffset = 0;
if (sbtable) { if (sbtable) {
currOffset = ROUNDUP_CL(currOffset); currOffset = ROUNDUP_CL(currOffset);
sbmatcherOffset = currOffset; proto.sbmatcherOffset = currOffset;
currOffset += verify_u32(sbsize); currOffset += verify_u32(sbsize);
} }
u32 leftOffset = ROUNDUP_N(currOffset, alignof(LeftNfaInfo)); currOffset = ROUNDUP_N(currOffset, alignof(LeftNfaInfo));
u32 roseLen = sizeof(LeftNfaInfo) * leftInfoTable.size(); proto.leftOffset = currOffset;
currOffset = leftOffset + roseLen; currOffset += sizeof(LeftNfaInfo) * leftInfoTable.size();
u32 lookaroundReachOffset = currOffset; proto.lookaroundReachOffset = currOffset;
u32 lookaroundReachLen = REACH_BITVECTOR_LEN * bc.lookaround.size(); currOffset += REACH_BITVECTOR_LEN * bc.lookaround.size();
currOffset = lookaroundReachOffset + lookaroundReachLen;
u32 lookaroundTableOffset = currOffset; proto.lookaroundTableOffset = currOffset;
u32 lookaroundTableLen = sizeof(s8) * bc.lookaround.size(); currOffset += sizeof(s8) * bc.lookaround.size();
currOffset = lookaroundTableOffset + lookaroundTableLen;
u32 nfaInfoOffset = ROUNDUP_N(currOffset, sizeof(u32)); currOffset = ROUNDUP_N(currOffset, sizeof(u32));
u32 nfaInfoLen = sizeof(NfaInfo) * queue_count; proto.nfaInfoOffset = currOffset;
currOffset = nfaInfoOffset + nfaInfoLen; currOffset += sizeof(NfaInfo) * queue_count;
currOffset = ROUNDUP_N(currOffset, alignof(mmbit_sparse_iter)); auto activeLeftIter = buildActiveLeftIter(leftInfoTable);
u32 activeLeftIterOffset = currOffset; if (!activeLeftIter.empty()) {
currOffset += activeLeftIter.size() * sizeof(mmbit_sparse_iter); currOffset = ROUNDUP_N(currOffset, alignof(mmbit_sparse_iter));
proto.activeLeftIterOffset = currOffset;
currOffset += activeLeftIter.size() * sizeof(mmbit_sparse_iter);
}
u32 activeArrayCount = leftfixBeginQueue; proto.activeArrayCount = proto.leftfixBeginQueue;
u32 activeLeftCount = leftInfoTable.size(); proto.activeLeftCount = verify_u32(leftInfoTable.size());
u32 rosePrefixCount = countRosePrefixes(leftInfoTable); proto.rosePrefixCount = countRosePrefixes(leftInfoTable);
u32 rev_nfa_table_offset; u32 rev_nfa_table_offset;
vector<u32> rev_nfa_offsets; vector<u32> rev_nfa_offsets;
prepSomRevNfas(ssm, &rev_nfa_table_offset, &rev_nfa_offsets, &currOffset); prepSomRevNfas(ssm, &rev_nfa_table_offset, &rev_nfa_offsets, &currOffset);
// Build engine header and copy tables into place. proto.anchorStateSize = atable ? anchoredStateSize(*atable) : 0;
u32 anchorStateSize = atable ? anchoredStateSize(*atable) : 0;
DEBUG_PRINTF("rose history required %zu\n", historyRequired); DEBUG_PRINTF("rose history required %zu\n", historyRequired);
assert(!cc.streaming || historyRequired <= cc.grey.maxHistoryAvailable); assert(!cc.streaming || historyRequired <= cc.grey.maxHistoryAvailable);
@ -5516,169 +5514,122 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
assert(!cc.streaming || historyRequired <= assert(!cc.streaming || historyRequired <=
max(cc.grey.maxHistoryAvailable, cc.grey.somMaxRevNfaLength)); max(cc.grey.maxHistoryAvailable, cc.grey.somMaxRevNfaLength));
RoseStateOffsets stateOffsets; fillStateOffsets(*this, bc.numStates, proto.anchorStateSize,
memset(&stateOffsets, 0, sizeof(stateOffsets)); proto.activeArrayCount, proto.activeLeftCount,
fillStateOffsets(*this, bc.numStates, anchorStateSize, laggedRoseCount, longLitStreamStateRequired,
activeArrayCount, activeLeftCount, laggedRoseCount, historyRequired, &proto.stateOffsets);
longLitStreamStateRequired, historyRequired,
&stateOffsets);
scatter_plan_raw state_scatter; scatter_plan_raw state_scatter = buildStateScatterPlan(
buildStateScatterPlan(sizeof(u8), bc.numStates, sizeof(u8), bc.numStates, proto.activeLeftCount, proto.rosePrefixCount,
activeLeftCount, rosePrefixCount, stateOffsets, proto.stateOffsets, cc.streaming, proto.activeArrayCount,
cc.streaming, activeArrayCount, outfixBeginQueue, proto.outfixBeginQueue, proto.outfixEndQueue);
outfixEndQueue, &state_scatter);
currOffset = ROUNDUP_N(currOffset, alignof(scatter_unit_u64a)); currOffset = ROUNDUP_N(currOffset, alignof(scatter_unit_u64a));
u32 state_scatter_aux_offset = currOffset; u32 state_scatter_aux_offset = currOffset;
currOffset += aux_size(state_scatter); currOffset += aux_size(state_scatter);
currOffset = ROUNDUP_N(currOffset, alignof(ReportID)); currOffset = ROUNDUP_N(currOffset, alignof(ReportID));
u32 dkeyOffset = currOffset; proto.invDkeyOffset = currOffset;
currOffset += rm.numDkeys() * sizeof(ReportID); currOffset += rm.numDkeys() * sizeof(ReportID);
aligned_unique_ptr<RoseEngine> engine proto.historyRequired = verify_u32(historyRequired);
= aligned_zmalloc_unique<RoseEngine>(currOffset); proto.ekeyCount = rm.numEkeys();
assert(engine); // will have thrown bad_alloc otherwise. proto.dkeyCount = rm.numDkeys();
char *ptr = (char *)engine.get(); proto.dkeyLogSize = fatbit_size(proto.dkeyCount);
assert(ISALIGNED_CL(ptr));
if (atable) { proto.somHorizon = ssm.somPrecision();
assert(amatcherOffset); proto.somLocationCount = ssm.numSomSlots();
memcpy(ptr + amatcherOffset, atable.get(), asize); proto.somLocationFatbitSize = fatbit_size(proto.somLocationCount);
}
if (ftable) {
assert(fmatcherOffset);
memcpy(ptr + fmatcherOffset, ftable.get(), fsize);
}
if (drtable) {
assert(drmatcherOffset);
memcpy(ptr + drmatcherOffset, drtable.get(), drsize);
}
if (etable) {
assert(ematcherOffset);
memcpy(ptr + ematcherOffset, etable.get(), esize);
}
if (sbtable) {
assert(sbmatcherOffset);
memcpy(ptr + sbmatcherOffset, sbtable.get(), sbsize);
}
memcpy(&engine->stateOffsets, &stateOffsets, sizeof(stateOffsets)); proto.needsCatchup = bc.needs_catchup ? 1 : 0;
engine->historyRequired = verify_u32(historyRequired); proto.runtimeImpl = pickRuntimeImpl(*this, bc, proto.outfixEndQueue);
proto.mpvTriggeredByLeaf = anyEndfixMpvTriggers(*this);
engine->ekeyCount = rm.numEkeys(); proto.queueCount = queue_count;
engine->dkeyCount = rm.numDkeys(); proto.activeQueueArraySize = fatbit_size(queue_count);
engine->dkeyLogSize = fatbit_size(engine->dkeyCount); proto.handledKeyCount = prog_build.handledKeys.size();
engine->invDkeyOffset = dkeyOffset; proto.handledKeyFatbitSize = fatbit_size(proto.handledKeyCount);
copy_bytes(ptr + dkeyOffset, rm.getDkeyToReportTable());
engine->somHorizon = ssm.somPrecision(); proto.rolesWithStateCount = bc.numStates;
engine->somLocationCount = ssm.numSomSlots();
engine->somLocationFatbitSize = fatbit_size(engine->somLocationCount);
engine->needsCatchup = bc.needs_catchup ? 1 : 0; proto.roseCount = verify_u32(leftInfoTable.size());
proto.initMpvNfa = mpv_as_outfix ? 0 : MO_INVALID_IDX;
proto.stateSize = mmbit_size(bc.numStates);
engine->reportProgramOffset = reportProgramOffset; proto.delay_fatbit_size = fatbit_size(proto.delay_count);
engine->reportProgramCount = reportProgramCount; proto.anchored_fatbit_size = fatbit_size(proto.anchored_count);
engine->delayProgramOffset = delayProgramOffset;
engine->anchoredProgramOffset = anchoredProgramOffset;
engine->runtimeImpl = pickRuntimeImpl(*this, bc, outfixEndQueue);
engine->mpvTriggeredByLeaf = anyEndfixMpvTriggers(*this);
engine->activeArrayCount = activeArrayCount;
engine->activeLeftCount = activeLeftCount;
engine->queueCount = queue_count;
engine->activeQueueArraySize = fatbit_size(queue_count);
engine->eagerIterOffset = eagerIterOffset;
engine->handledKeyCount = prog_build.handledKeys.size();
engine->handledKeyFatbitSize = fatbit_size(engine->handledKeyCount);
engine->rolesWithStateCount = bc.numStates;
engine->leftOffset = leftOffset;
engine->roseCount = verify_u32(leftInfoTable.size());
engine->lookaroundTableOffset = lookaroundTableOffset;
engine->lookaroundReachOffset = lookaroundReachOffset;
engine->outfixBeginQueue = outfixBeginQueue;
engine->outfixEndQueue = outfixEndQueue;
engine->leftfixBeginQueue = leftfixBeginQueue;
engine->initMpvNfa = mpv_as_outfix ? 0 : MO_INVALID_IDX;
engine->stateSize = mmbit_size(bc.numStates);
engine->anchorStateSize = anchorStateSize;
engine->nfaInfoOffset = nfaInfoOffset;
engine->eodProgramOffset = eodProgramOffset;
engine->lastByteHistoryIterOffset = lastByteOffset;
engine->delay_count = delayProgramCount;
engine->delay_fatbit_size = fatbit_size(engine->delay_count);
engine->anchored_count = anchoredProgramCount;
engine->anchored_fatbit_size = fatbit_size(engine->anchored_count);
engine->rosePrefixCount = rosePrefixCount;
engine->activeLeftIterOffset
= activeLeftIter.empty() ? 0 : activeLeftIterOffset;
// Set scanning mode.
if (!cc.streaming) {
engine->mode = HS_MODE_BLOCK;
} else if (cc.vectored) {
engine->mode = HS_MODE_VECTORED;
} else {
engine->mode = HS_MODE_STREAM;
}
// The Small Write matcher is (conditionally) added to the RoseEngine in // The Small Write matcher is (conditionally) added to the RoseEngine in
// another pass by the caller. Set to zero (meaning no SMWR engine) for // another pass by the caller. Set to zero (meaning no SMWR engine) for
// now. // now.
engine->smallWriteOffset = 0; proto.smallWriteOffset = 0;
engine->amatcherOffset = amatcherOffset; proto.amatcherMinWidth = findMinWidth(*this, ROSE_ANCHORED);
engine->ematcherOffset = ematcherOffset; proto.fmatcherMinWidth = findMinWidth(*this, ROSE_FLOATING);
engine->sbmatcherOffset = sbmatcherOffset; proto.eodmatcherMinWidth = findMinWidth(*this, ROSE_EOD_ANCHORED);
engine->fmatcherOffset = fmatcherOffset; proto.amatcherMaxBiAnchoredWidth = findMaxBAWidth(*this, ROSE_ANCHORED);
engine->drmatcherOffset = drmatcherOffset; proto.fmatcherMaxBiAnchoredWidth = findMaxBAWidth(*this, ROSE_FLOATING);
engine->longLitTableOffset = longLitTableOffset; proto.minWidth = hasBoundaryReports(boundary) ? 0 : minWidth;
engine->amatcherMinWidth = findMinWidth(*this, ROSE_ANCHORED); proto.minWidthExcludingBoundaries = minWidth;
engine->fmatcherMinWidth = findMinWidth(*this, ROSE_FLOATING); proto.floatingMinLiteralMatchOffset = bc.floatingMinLiteralMatchOffset;
engine->eodmatcherMinWidth = findMinWidth(*this, ROSE_EOD_ANCHORED);
engine->amatcherMaxBiAnchoredWidth = findMaxBAWidth(*this, ROSE_ANCHORED);
engine->fmatcherMaxBiAnchoredWidth = findMaxBAWidth(*this, ROSE_FLOATING);
engine->size = currOffset;
engine->minWidth = hasBoundaryReports(boundary) ? 0 : minWidth;
engine->minWidthExcludingBoundaries = minWidth;
engine->floatingMinLiteralMatchOffset = bc.floatingMinLiteralMatchOffset;
engine->maxBiAnchoredWidth = findMaxBAWidth(*this); proto.maxBiAnchoredWidth = findMaxBAWidth(*this);
engine->noFloatingRoots = hasNoFloatingRoots(); proto.noFloatingRoots = hasNoFloatingRoots();
engine->requiresEodCheck = hasEodAnchors(*this, bc, outfixEndQueue); proto.requiresEodCheck = hasEodAnchors(*this, bc, proto.outfixEndQueue);
engine->hasOutfixesInSmallBlock = hasNonSmallBlockOutfix(outfixes); proto.hasOutfixesInSmallBlock = hasNonSmallBlockOutfix(outfixes);
engine->canExhaust = rm.patternSetCanExhaust(); proto.canExhaust = rm.patternSetCanExhaust();
engine->hasSom = hasSom; proto.hasSom = hasSom;
/* populate anchoredDistance, floatingDistance, floatingMinDistance, etc */ /* populate anchoredDistance, floatingDistance, floatingMinDistance, etc */
fillMatcherDistances(*this, engine.get()); fillMatcherDistances(*this, &proto);
engine->initialGroups = getInitialGroups(); proto.initialGroups = getInitialGroups();
engine->floating_group_mask = fgroups; proto.floating_group_mask = fgroups;
engine->totalNumLiterals = verify_u32(literal_info.size()); proto.totalNumLiterals = verify_u32(literal_info.size());
engine->asize = verify_u32(asize); proto.asize = verify_u32(asize);
engine->ematcherRegionSize = ematcher_region_size; proto.ematcherRegionSize = ematcher_region_size;
engine->longLitStreamState = verify_u32(longLitStreamStateRequired); proto.longLitStreamState = verify_u32(longLitStreamStateRequired);
engine->boundary.reportEodOffset = boundary_out.reportEodOffset; proto.size = currOffset;
engine->boundary.reportZeroOffset = boundary_out.reportZeroOffset;
engine->boundary.reportZeroEodOffset = boundary_out.reportZeroEodOffset; // Time to allocate the real RoseEngine structure.
auto engine = aligned_zmalloc_unique<RoseEngine>(currOffset);
assert(engine); // will have thrown bad_alloc otherwise.
// Copy in our prototype engine data.
memcpy(engine.get(), &proto, sizeof(proto));
char *ptr = (char *)engine.get();
assert(ISALIGNED_CL(ptr));
if (atable) {
assert(proto.amatcherOffset);
memcpy(ptr + proto.amatcherOffset, atable.get(), asize);
}
if (ftable) {
assert(proto.fmatcherOffset);
memcpy(ptr + proto.fmatcherOffset, ftable.get(), fsize);
}
if (drtable) {
assert(proto.drmatcherOffset);
memcpy(ptr + proto.drmatcherOffset, drtable.get(), drsize);
}
if (etable) {
assert(proto.ematcherOffset);
memcpy(ptr + proto.ematcherOffset, etable.get(), esize);
}
if (sbtable) {
assert(proto.sbmatcherOffset);
memcpy(ptr + proto.sbmatcherOffset, sbtable.get(), sbsize);
}
copy_bytes(ptr + proto.invDkeyOffset, rm.getDkeyToReportTable());
write_out(&engine->state_init, (char *)engine.get(), state_scatter, write_out(&engine->state_init, (char *)engine.get(), state_scatter,
state_scatter_aux_offset); state_scatter_aux_offset);
NfaInfo *nfa_infos = (NfaInfo *)(ptr + nfaInfoOffset); NfaInfo *nfa_infos = (NfaInfo *)(ptr + proto.nfaInfoOffset);
populateNfaInfoBasics(*this, bc, outfixes, suffixEkeyLists, populateNfaInfoBasics(*this, bc, outfixes, suffixEkeyLists,
no_retrigger_queues, nfa_infos); no_retrigger_queues, nfa_infos);
updateNfaState(bc, &engine->stateOffsets, nfa_infos, updateNfaState(bc, &engine->stateOffsets, nfa_infos,
@ -5689,8 +5640,8 @@ aligned_unique_ptr<RoseEngine> RoseBuildImpl::buildFinalEngine(u32 minWidth) {
bc.engine_blob.write_bytes(engine.get()); bc.engine_blob.write_bytes(engine.get());
copy_bytes(ptr + engine->leftOffset, leftInfoTable); copy_bytes(ptr + engine->leftOffset, leftInfoTable);
fillLookaroundTables(ptr + lookaroundTableOffset, fillLookaroundTables(ptr + proto.lookaroundTableOffset,
ptr + lookaroundReachOffset, bc.lookaround); ptr + proto.lookaroundReachOffset, bc.lookaround);
fillInSomRevNfas(engine.get(), ssm, rev_nfa_table_offset, rev_nfa_offsets); fillInSomRevNfas(engine.get(), ssm, rev_nfa_table_offset, rev_nfa_offsets);
copy_bytes(ptr + engine->activeLeftIterOffset, activeLeftIter); copy_bytes(ptr + engine->activeLeftIterOffset, activeLeftIter);

View File

@ -1,5 +1,5 @@
/* /*
* Copyright (c) 2015, Intel Corporation * Copyright (c) 2015-2017, Intel Corporation
* *
* Redistribution and use in source and binary forms, with or without * Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met: * modification, are permitted provided that the following conditions are met:
@ -63,24 +63,24 @@ void merge_in(scatter_plan_raw *out, const scatter_plan_raw &in) {
insert(&out->p_u8, out->p_u8.end(), in.p_u8); insert(&out->p_u8, out->p_u8.end(), in.p_u8);
} }
void buildStateScatterPlan(u32 role_state_offset, u32 role_state_count, scatter_plan_raw buildStateScatterPlan(u32 role_state_offset,
u32 left_array_count, u32 left_prefix_count, u32 role_state_count, u32 left_array_count, u32 left_prefix_count,
const RoseStateOffsets &stateOffsets, const RoseStateOffsets &stateOffsets, bool streaming,
bool streaming, u32 leaf_array_count, u32 leaf_array_count, u32 outfix_begin, u32 outfix_end) {
u32 outfix_begin, u32 outfix_end, scatter_plan_raw out;
scatter_plan_raw *out) {
/* init role array */ /* init role array */
scatter_plan_raw spr_role; scatter_plan_raw spr_role;
mmbBuildClearPlan(role_state_count, &spr_role); mmbBuildClearPlan(role_state_count, &spr_role);
rebase(&spr_role, role_state_offset); rebase(&spr_role, role_state_offset);
merge_in(out, spr_role); merge_in(&out, spr_role);
/* init rose array: turn on prefixes */ /* init rose array: turn on prefixes */
u32 rose_array_offset = stateOffsets.activeLeftArray; u32 rose_array_offset = stateOffsets.activeLeftArray;
scatter_plan_raw spr_rose; scatter_plan_raw spr_rose;
mmbBuildInitRangePlan(left_array_count, 0, left_prefix_count, &spr_rose); mmbBuildInitRangePlan(left_array_count, 0, left_prefix_count, &spr_rose);
rebase(&spr_rose, rose_array_offset); rebase(&spr_rose, rose_array_offset);
merge_in(out, spr_rose); merge_in(&out, spr_rose);
/* suffix/outfix array */ /* suffix/outfix array */
scatter_plan_raw spr_leaf; scatter_plan_raw spr_leaf;
@ -91,7 +91,9 @@ void buildStateScatterPlan(u32 role_state_offset, u32 role_state_count,
mmbBuildClearPlan(leaf_array_count, &spr_leaf); mmbBuildClearPlan(leaf_array_count, &spr_leaf);
} }
rebase(&spr_leaf, stateOffsets.activeLeafArray); rebase(&spr_leaf, stateOffsets.activeLeafArray);
merge_in(out, spr_leaf); merge_in(&out, spr_leaf);
return out;
} }
u32 aux_size(const scatter_plan_raw &raw) { u32 aux_size(const scatter_plan_raw &raw) {

View File

@ -1,5 +1,5 @@
/* /*
* Copyright (c) 2015, Intel Corporation * Copyright (c) 2015-2017, Intel Corporation
* *
* Redistribution and use in source and binary forms, with or without * Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met: * modification, are permitted provided that the following conditions are met:
@ -45,12 +45,10 @@ struct scatter_plan_raw {
std::vector<scatter_unit_u8> p_u8; std::vector<scatter_unit_u8> p_u8;
}; };
void buildStateScatterPlan(u32 role_state_offset, u32 role_state_count, scatter_plan_raw buildStateScatterPlan(u32 role_state_offset,
u32 left_array_count, u32 left_prefix_count, u32 role_state_count, u32 left_array_count, u32 left_prefix_count,
const RoseStateOffsets &stateOffsets, const RoseStateOffsets &stateOffsets, bool streaming,
bool streaming, u32 leaf_array_count, u32 leaf_array_count, u32 outfix_begin, u32 outfix_end);
u32 outfix_begin, u32 outfix_end,
scatter_plan_raw *out);
u32 aux_size(const scatter_plan_raw &raw); u32 aux_size(const scatter_plan_raw &raw);