private static void initializeScan()

in phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java [199:367]


    private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer offset, Scan scan) throws SQLException {
        StatementContext context = plan.getContext();
        TableRef tableRef = plan.getTableRef();
        boolean wildcardIncludesDynamicCols = context.getConnection().getQueryServices()
                .getConfiguration().getBoolean(WILDCARD_QUERY_DYNAMIC_COLS_ATTRIB,
                        DEFAULT_WILDCARD_QUERY_DYNAMIC_COLS_ATTRIB);
        PTable table = tableRef.getTable();

        // If the table has Conditional TTL set, then we need to add all the non PK columns
        // referenced in the conditional TTL expression to the scan. This can influence the
        // filters that are applied to the scan so do this before the filter analysis.
        if (table.hasConditionalTTL()) {
            CompiledConditionalTTLExpression ttlExpr =
                    (CompiledConditionalTTLExpression)
                            table.getCompiledTTLExpression(context.getConnection());
            Set<ColumnReference> colsReferenced = ttlExpr.getColumnsReferenced();
            for (ColumnReference colref : colsReferenced) {
                // adding the ttl expr columns to the where condition columns ensures that
                // the ttl expr columns are correctly added to the scan
                context.addWhereConditionColumn(colref.getFamily(), colref.getQualifier());
            }
        }

        Map<byte [], NavigableSet<byte []>> familyMap = scan.getFamilyMap();
        // Hack for PHOENIX-2067 to force raw scan over all KeyValues to fix their row keys
        if (context.getConnection().isDescVarLengthRowKeyUpgrade()) {
            // We project *all* KeyValues across all column families as we make a pass over
            // a physical table and we want to make sure we catch all KeyValues that may be
            // dynamic or part of an updatable view.
            familyMap.clear();
            scan.readAllVersions();
            scan.setFilter(null); // Remove any filter
            scan.setRaw(true); // Traverse (and subsequently clone) all KeyValues
            // Pass over PTable so we can re-write rows according to the row key schema
            scan.setAttribute(BaseScannerRegionObserverConstants.UPGRADE_DESC_ROW_KEY, UngroupedAggregateRegionObserverHelper.serialize(table));
        } else {
            FilterableStatement statement = plan.getStatement();
            RowProjector projector = plan.getProjector();
            boolean optimizeProjection = false;
            boolean keyOnlyFilter = familyMap.isEmpty() && !wildcardIncludesDynamicCols &&
                    context.getWhereConditionColumns().isEmpty();
            if (!projector.projectEverything()) {
                // If nothing projected into scan and we only have one column family, just allow everything
                // to be projected and use a FirstKeyOnlyFilter to skip from row to row. This turns out to
                // be quite a bit faster.
                // Where condition columns also will get added into familyMap
                // When where conditions are present, we cannot add FirstKeyOnlyFilter at beginning.
                // FIXME: we only enter this if the number of column families is 1 because otherwise
                // local indexes break because it appears that the column families in the PTable do
                // not match the actual column families of the table (which is bad).
                if (keyOnlyFilter && table.getColumnFamilies().size() == 1) {
                    // Project the one column family. We must project a column family since it's possible
                    // that there are other non declared column families that we need to ignore.
                    scan.addFamily(table.getColumnFamilies().get(0).getName().getBytes());
                } else {
                    optimizeProjection = true;
                    if (projector.projectEveryRow()) {
                        if (table.getViewType() == ViewType.MAPPED) {
                            // Since we don't have the empty key value in MAPPED tables, 
                            // we must project all CFs in HRS. However, only the
                            // selected column values are returned back to client.
                            context.getWhereConditionColumns().clear();
                            for (PColumnFamily family : table.getColumnFamilies()) {
                                context.addWhereConditionColumn(family.getName().getBytes(), null);
                            }
                        } else {
                            byte[] ecf = SchemaUtil.getEmptyColumnFamily(table);
                            // Project empty key value unless the column family containing it has
                            // been projected in its entirety.
                            if (!familyMap.containsKey(ecf) || familyMap.get(ecf) != null) {
                                scan.addColumn(ecf, EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst());
                            }
                        }
                    }
                }
            } else {
                boolean containsNullableGroubBy = false;
                if (!plan.getOrderBy().isEmpty()) {
                    for (OrderByExpression orderByExpression : plan.getOrderBy()
                            .getOrderByExpressions()) {
                        if (orderByExpression.getExpression().isNullable()) {
                            containsNullableGroubBy = true;
                            break;
                        }
                    }
                }
                if (containsNullableGroubBy) {
                    byte[] ecf = SchemaUtil.getEmptyColumnFamily(table);
                    if (!familyMap.containsKey(ecf) || familyMap.get(ecf) != null) {
                        scan.addColumn(ecf, EncodedColumnsUtil.getEmptyKeyValueInfo(table)
                                .getFirst());
                    }
                }
            }
            // Add FirstKeyOnlyFilter or EmptyColumnOnlyFilter if there are no references
            // to key value columns. We use FirstKeyOnlyFilter when possible
            if (keyOnlyFilter) {
                byte[] ecf = SchemaUtil.getEmptyColumnFamily(table);
                byte[] ecq = table.getEncodingScheme() == NON_ENCODED_QUALIFIERS ?
                        QueryConstants.EMPTY_COLUMN_BYTES :
                        table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
                if (table.getEncodingScheme() == NON_ENCODED_QUALIFIERS) {
                    ScanUtil.andFilterAtBeginning(scan, new EmptyColumnOnlyFilter(ecf, ecq));
                } else  if (table.getColumnFamilies().size() == 0) {
                    ScanUtil.andFilterAtBeginning(scan, new FirstKeyOnlyFilter());
                } else {
                    // There are more than column families. If the empty column family is the
                    // first column family lexicographically then FirstKeyOnlyFilter would return
                    // the empty column
                    List<byte[]> families = new ArrayList<>(table.getColumnFamilies().size());
                    for (PColumnFamily family : table.getColumnFamilies()) {
                        families.add(family.getName().getBytes());
                    }
                    Collections.sort(families, Bytes.BYTES_COMPARATOR);
                    byte[] firstFamily = families.get(0);
                    if (Bytes.compareTo(ecf, 0, ecf.length,
                            firstFamily, 0, firstFamily.length) == 0) {
                        ScanUtil.andFilterAtBeginning(scan, new FirstKeyOnlyFilter());
                    } else {
                        ScanUtil.andFilterAtBeginning(scan, new EmptyColumnOnlyFilter(ecf, ecq));
                    }
                }
            }

            if (perScanLimit != null) {
                if (scan.getAttribute(BaseScannerRegionObserverConstants.INDEX_FILTER) == null) {
                    ScanUtil.andFilterAtEnd(scan, new PageFilter(perScanLimit));
                } else {
                    // if we have an index filter and a limit, handle the limit after the filter
                    // we cast the limit to a long even though it passed as an Integer so that
                    // if we need extend this in the future the serialization is unchanged
                    scan.setAttribute(BaseScannerRegionObserverConstants.INDEX_LIMIT,
                            Bytes.toBytes((long) perScanLimit));
                }
            }

            if (offset != null) {
                ScanUtil.addOffsetAttribute(scan, offset);
            }
            GroupBy groupBy = plan.getGroupBy();
            int cols = groupBy.getOrderPreservingColumnCount();
            if (cols > 0 && keyOnlyFilter &&
                !plan.getStatement().getHint().hasHint(HintNode.Hint.RANGE_SCAN) &&
                cols < plan.getTableRef().getTable().getRowKeySchema().getFieldCount() &&
                groupBy.isOrderPreserving() &&
                (context.getAggregationManager().isEmpty() || groupBy.isUngroupedAggregate())) {

                    ScanUtil.andFilterAtEnd(scan,
                            new DistinctPrefixFilter(plan.getTableRef().getTable().getRowKeySchema(),cols));
                    if (!groupBy.isUngroupedAggregate() && plan.getLimit() != null) {
                        // We can push the limit to the server,but for UngroupedAggregate
                        // we can not push the limit.
                        ScanUtil.andFilterAtEnd(scan, new PageFilter(plan.getLimit()));
                    }
            }
            scan.setAttribute(BaseScannerRegionObserverConstants.QUALIFIER_ENCODING_SCHEME, new byte[]{table.getEncodingScheme().getSerializedMetadataValue()});
            scan.setAttribute(BaseScannerRegionObserverConstants.IMMUTABLE_STORAGE_ENCODING_SCHEME, new byte[]{table.getImmutableStorageScheme().getSerializedMetadataValue()});
            // we use this flag on the server side to determine which value column qualifier to use in the key value we return from server.
            scan.setAttribute(BaseScannerRegionObserverConstants.USE_NEW_VALUE_COLUMN_QUALIFIER, Bytes.toBytes(true));
            // When analyzing the table, there is no look up for key values being done.
            // So there is no point setting the range.
            if (!ScanUtil.isAnalyzeTable(scan)) {
                setQualifierRanges(keyOnlyFilter, table, scan, context);
            }
            if (optimizeProjection) {
                optimizeProjection(context, scan, table, statement);
            }
        }
    }