http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 8820ff7,40f3f33..edf8e47
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -181,8 -247,11 +181,11 @@@ public class SelectStatement implement
  
      private Pageable getPageableCommand(QueryOptions options, int limit, long 
now) throws RequestValidationException
      {
 -        if (isNotReturningAnyRows(options))
++        if (restrictions.isNotReturningAnyRows(options))
+             return null;
+ 
          int limitForQuery = updateLimitForQuery(limit);
 -        if (isKeyRange || usesSecondaryIndexing)
 +        if (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())
              return getRangeCommand(options, limitForQuery, now);
  
          List<ReadCommand> commands = getSliceCommands(options, limitForQuery, 
now);
@@@ -596,52 -961,338 +599,58 @@@
          }
      }
  
 -    /** Returns true if a non-frozen collection is selected, false otherwise. 
*/
 -    private boolean selectACollection()
 +    /**
 +     * May be used by custom QueryHandler implementations
 +     */
 +    public List<IndexExpression> getValidatedIndexExpressions(QueryOptions 
options) throws InvalidRequestException
      {
 -        if (!cfm.comparator.hasCollections())
 -            return false;
 +        if (!restrictions.usesSecondaryIndexing())
 +            return Collections.emptyList();
  
 -        for (ColumnDefinition def : selection.getColumns())
 -        {
 -            if (def.type.isCollection() && def.type.isMultiCell())
 -                return true;
 -        }
 +        ColumnFamilyStore cfs = 
Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
 +        SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
  
 -        return false;
 +        List<IndexExpression> expressions = 
restrictions.getIndexExpressions(secondaryIndexManager, options);
 +        secondaryIndexManager.validateIndexSearchersForQuery(expressions);
 +
 +        return expressions;
      }
  
 -    @VisibleForTesting
 -    static List<Composite> buildBound(Bound bound,
 -                                      List<ColumnDefinition> defs,
 -                                      Restriction[] restrictions,
 -                                      boolean isReversed,
 -                                      CType type,
 -                                      QueryOptions options) throws 
InvalidRequestException
 +    private CellName makeExclusiveSliceBound(Bound bound, CellNameType type, 
QueryOptions options) throws InvalidRequestException
      {
 -        CBuilder builder = type.builder();
 -
 -        // The end-of-component of composite doesn't depend on whether the
 -        // component type is reversed or not (i.e. the ReversedType is applied
 -        // to the component comparator but not to the end-of-component 
itself),
 -        // it only depends on whether the slice is reversed
 -        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
 -        for (int i = 0, m = defs.size(); i < m; i++)
 -        {
 -            ColumnDefinition def = defs.get(i);
 -
 -            // In a restriction, we always have Bound.START < Bound.END for 
the "base" comparator.
 -            // So if we're doing a reverse slice, we must inverse the bounds 
when giving them as start and end of the slice filter.
 -            // But if the actual comparator itself is reversed, we must 
inversed the bounds too.
 -            Bound b = isReversed == isReversedType(def) ? bound : 
Bound.reverse(bound);
 -            Restriction r = restrictions[def.position()];
 -            if (isNullRestriction(r, b) || !r.canEvaluateWithSlices())
 -            {
 -                // There wasn't any non EQ relation on that key, we select 
all records having the preceding component as prefix.
 -                // For composites, if there was preceding component and we're 
computing the end, we must change the last component
 -                // End-Of-Component, otherwise we would be selecting only one 
record.
 -                Composite prefix = builder.build();
 -                return Collections.singletonList(eocBound == Bound.END ? 
prefix.end() : prefix.start());
 -            }
 -            if (r.isSlice())
 -            {
 -                if (r.isMultiColumn())
 -                {
 -                    MultiColumnRestriction.Slice slice = 
(MultiColumnRestriction.Slice) r;
 +        // clusteringColumnBounds may reverse bound if clustering order is 
reversed
 +        // but areRequestedBoundsInclusive checks for 
Restriction::isInclusive and never
 +        // reverses the order. In order to avoid inconsistencies and check 
inclusive
 +        // bounds correctly, we need to check for column order and reverse 
it. See CASSANDRA-10988
 +        if 
(restrictions.areRequestedBoundsInclusive(reverseBoundIfNeeded(bound)))
 +            return null;
  
-        return 
type.makeCellName(restrictions.getClusteringColumnsBounds(bound, 
options).get(0));
 -                    if (!slice.hasBound(b))
 -                    {
 -                        Composite prefix = builder.build();
 -                        return 
Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
 -                                ? prefix.end()
 -                                : prefix);
 -                    }
++        // We can only reach that if the table is dense non-compound tables.
++        // By consequence, we know that the table is a COMPACT table with 
only one clustering column.
++        ByteBuffer value = restrictions.getClusteringColumnsBounds(bound, 
options).get(0);
+ 
 -                    List<ByteBuffer> vals = slice.componentBounds(b, options);
 -
 -                    for (int j = 0, n = vals.size(); j < n; j++)
 -                        addValue(builder, defs.get(i + j), vals.get(j)) ;
 -                }
 -                else
 -                {
 -                    builder.add(getSliceValue(r, b, options));
 -                }
 -                Operator relType = 
((Restriction.Slice)r).getRelation(eocBound, b);
 -                return 
Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
 -            }
 -
 -            if (r.isIN())
 -            {
 -                // The IN query might not have listed the values in 
comparator order, so we need to re-sort
 -                // the bounds lists to make sure the slices works correctly 
(also, to avoid duplicates).
 -                TreeSet<Composite> inValues = new TreeSet<>(isReversed ? 
type.reverseComparator() : type);
 -
 -                if (r.isMultiColumn())
 -                {
 -                    List<List<ByteBuffer>> splitInValues = 
((MultiColumnRestriction.IN) r).splitValues(options);
 -
 -                    for (List<ByteBuffer> components : splitInValues)
 -                    {
 -                        for (int j = 0; j < components.size(); j++)
 -                            if (components.get(j) == null)
 -                                throw new InvalidRequestException("Invalid 
null value in condition for column " + defs.get(i + j).name);
 -
 -                        Composite prefix = builder.buildWith(components);
 -                        inValues.add(builder.remainingCount() == 0 ? prefix : 
addEOC(prefix, eocBound));
 -                    }
 -                    return new ArrayList<>(inValues);
 -                }
 -
 -                List<ByteBuffer> values = r.values(options);
 -                if (values.size() != 1)
 -                {
 -                    // IN query, we only support it on the clustering columns
 -                    assert def.position() == defs.size() - 1;
 -                    for (ByteBuffer val : values)
 -                    {
 -                        if (val == null)
 -                            throw new 
InvalidRequestException(String.format("Invalid null value in condition for 
column %s",
 -                                                                            
def.name));
 -                        Composite prefix = builder.buildWith(val);
 -                        // See below for why this
 -                        inValues.add(builder.remainingCount() == 0 ? prefix : 
addEOC(prefix, eocBound));
 -                    }
 -                    return new ArrayList<>(inValues);
 -                }
 -            }
 -
 -            List<ByteBuffer> values = r.values(options);
 -
 -            if (r.isMultiColumn())
 -            {
 -                for (int j = 0; j < values.size(); j++)
 -                    addValue(builder, defs.get(i + j), values.get(j));
 -                i += values.size() - 1; // skips the processed columns
 -            }
 -            else
 -            {
 -                addValue(builder, def, values.get(0));
 -            }
 -        }
 -        // Means no relation at all or everything was an equal
 -        // Note: if the builder is "full", there is no need to use the 
end-of-component bit. For columns selection,
 -        // it would be harmless to do it. However, we use this method got the 
partition key too. And when a query
 -        // with 2ndary index is done, and with the the partition provided 
with an EQ, we'll end up here, and in that
 -        // case using the eoc would be bad, since for the random partitioner 
we have no guarantee that
 -        // prefix.end() will sort after prefix (see #5240).
 -        Composite prefix = builder.build();
 -        return Collections.singletonList(builder.remainingCount() == 0 ? 
prefix : addEOC(prefix, eocBound));
 -    }
++        // Dense non-compound tables do not accept empty ByteBuffers. By 
consequence, if the slice value is empty
++        // we know that we can treat the slice as inclusive.
++        return value.hasRemaining() ? type.makeCellName(value) : null;
 +    }
  
      /**
 -     * Adds an EOC to the specified Composite.
 -     *
 -     * @param composite the composite
 -     * @param eocBound the EOC bound
 -     * @return a new <code>Composite</code> with the EOC corresponding to the 
eocBound
 +     * Reverses the specified bound if the non-compound clustering column is 
a reversed one.
 +     * @param bound bound to reverse
 +     * @return the bound reversed if the column type was a reversed one or 
the original bound
       */
 -    private static Composite addEOC(Composite composite, Bound eocBound)
 +    private Bound reverseBoundIfNeeded(Bound bound)
      {
 -        return eocBound == Bound.END ? composite.end() : composite.start();
 -    }
 +        assert !cfm.comparator.isCompound();
  
 -    /**
 -     * Adds the specified value to the specified builder
 -     *
 -     * @param builder the CBuilder to which the value must be added
 -     * @param def the column associated to the value
 -     * @param value the value to add
 -     * @throws InvalidRequestException if the value is null
 -     */
 -    private static void addValue(CBuilder builder, ColumnDefinition def, 
ByteBuffer value) throws InvalidRequestException
 -    {
 -        if (value == null)
 -            throw new InvalidRequestException(String.format("Invalid null 
value in condition for column %s", def.name));
 -        builder.add(value);
 -    }
 -
 -    private static Composite.EOC eocForRelation(Operator op)
 -    {
 -        switch (op)
 -        {
 -            case LT:
 -                // < X => using startOf(X) as finish bound
 -                return Composite.EOC.START;
 -            case GT:
 -            case LTE:
 -                // > X => using endOf(X) as start bound
 -                // <= X => using endOf(X) as finish bound
 -                return Composite.EOC.END;
 -            default:
 -                // >= X => using X as start bound (could use START_OF too)
 -                // = X => using X
 -                return Composite.EOC.NONE;
 -        }
 -    }
 -
 -    private static boolean isNullRestriction(Restriction r, Bound b)
 -    {
 -        return r == null || (r.isSlice() && 
!((Restriction.Slice)r).hasBound(b));
 -    }
 -
 -    private static ByteBuffer getSliceValue(Restriction r, Bound b, 
QueryOptions options) throws InvalidRequestException
 -    {
 -        Restriction.Slice slice = (Restriction.Slice)r;
 -        assert slice.hasBound(b);
 -        ByteBuffer val = slice.bound(b, options);
 -        if (val == null)
 -            throw new InvalidRequestException(String.format("Invalid null 
clustering key part %s", r));
 -        return val;
 -    }
 -
 -    private List<Composite> getRequestedBound(Bound b, QueryOptions options) 
throws InvalidRequestException
 -    {
 -        assert isColumnRange();
 -        List<Composite> bound = buildBound(b, cfm.clusteringColumns(), 
columnRestrictions, isReversed, cfm.comparator, options);
 -        for (Composite c : bound) {
 -            if (!c.isEmpty())
 -                QueryProcessor.validateComposite(c, cfm.comparator);
 -        }
 -        return bound;
 +        List<ColumnDefinition> columnDefs = cfm.clusteringColumns();
 +        return columnDefs.get(columnDefs.size() - 1).isReversedType() ? 
bound.reverse() : bound;
      }
  
 -    /**
 -     * May be used by custom QueryHandler implementations
 -     */
 -    public List<IndexExpression> getValidatedIndexExpressions(QueryOptions 
options) throws InvalidRequestException
 -    {
 -        if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
 -            return Collections.emptyList();
 -
 -        List<IndexExpression> expressions = new ArrayList<IndexExpression>();
 -        for (ColumnDefinition def : restrictedColumns.keySet())
 -        {
 -            Restriction restriction;
 -            switch (def.kind)
 -            {
 -                case PARTITION_KEY:
 -                    restriction = keyRestrictions[def.position()];
 -                    break;
 -                case CLUSTERING_COLUMN:
 -                    restriction = columnRestrictions[def.position()];
 -                    break;
 -                case REGULAR:
 -                case STATIC:
 -                    restriction = metadataRestrictions.get(def.name);
 -                    break;
 -                default:
 -                    // We don't allow restricting a COMPACT_VALUE for now in 
prepare.
 -                    throw new AssertionError();
 -            }
 -
 -            if (restriction.isSlice())
 -            {
 -                Restriction.Slice slice = (Restriction.Slice)restriction;
 -                for (Bound b : Bound.values())
 -                {
 -                    if (slice.hasBound(b))
 -                    {
 -                        ByteBuffer value = validateIndexedValue(def, 
slice.bound(b, options));
 -                        Operator op = slice.getIndexOperator(b);
 -                        // If the underlying comparator for name is reversed, 
we need to reverse the IndexOperator: user operation
 -                        // always refer to the "forward" sorting even if the 
clustering order is reversed, but the 2ndary code does
 -                        // use the underlying comparator as is.
 -                        if (def.type instanceof ReversedType)
 -                            op = reverse(op);
 -                        expressions.add(new IndexExpression(def.name.bytes, 
op, value));
 -                    }
 -                }
 -            }
 -            else if (restriction.isContains())
 -            {
 -                SingleColumnRestriction.Contains contains = 
(SingleColumnRestriction.Contains)restriction;
 -                for (ByteBuffer value : contains.values(options))
 -                {
 -                    validateIndexedValue(def, value);
 -                    expressions.add(new IndexExpression(def.name.bytes, 
Operator.CONTAINS, value));
 -                }
 -                for (ByteBuffer key : contains.keys(options))
 -                {
 -                    validateIndexedValue(def, key);
 -                    expressions.add(new IndexExpression(def.name.bytes, 
Operator.CONTAINS_KEY, key));
 -                }
 -            }
 -            else
 -            {
 -                ByteBuffer value;
 -                if (restriction.isMultiColumn())
 -                {
 -                    List<ByteBuffer> values = restriction.values(options);
 -                    value = values.get(def.position());
 -                }
 -                else
 -                {
 -                    List<ByteBuffer> values = restriction.values(options);
 -                    if (values.size() != 1)
 -                        throw new InvalidRequestException("IN restrictions 
are not supported on indexed columns");
 -
 -                    value = values.get(0);
 -                }
 -
 -                validateIndexedValue(def, value);
 -                expressions.add(new IndexExpression(def.name.bytes, 
Operator.EQ, value));
 -            }
 -        }
 -
 -        if (usesSecondaryIndexing)
 -        {
 -            ColumnFamilyStore cfs = 
Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
 -            SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
 -            secondaryIndexManager.validateIndexSearchersForQuery(expressions);
 -        }
 -        
 -        return expressions;
 -    }
 -
 -    private static ByteBuffer validateIndexedValue(ColumnDefinition def, 
ByteBuffer value) throws InvalidRequestException
 -    {
 -        if (value == null)
 -            throw new InvalidRequestException(String.format("Unsupported null 
value for indexed column %s", def.name));
 -        if (value.remaining() > 0xFFFF)
 -            throw new InvalidRequestException("Index expression values may 
not be larger than 64K");
 -        return value;
 -    }
 -
 -    private CellName makeExclusiveSliceBound(Bound bound, CellNameType type, 
QueryOptions options) throws InvalidRequestException
 -    {
 -        if (sliceRestriction.isInclusive(bound))
 -            return null;
 -
 -        // We can only reach that point if cfm.comparator.isCompound() = 
false and the table has some clustering columns.
 -        // By consequence, we know that the table is a COMPACT table with 
only one clustering column.
 -        ByteBuffer value = sliceRestriction.isMultiColumn() ? 
((MultiColumnRestriction.Slice) sliceRestriction).componentBounds(bound, 
options).get(0)
 -                                                            : 
sliceRestriction.bound(bound, options);
 -
 -        // Dense non-compound tables do not accept empty ByteBuffers. By 
consequence, if the slice value is empty
 -        // we know that we can treat the slice as inclusive.
 -        return value.hasRemaining() ? type.makeCellName(value) : null;
 -    }
 -
 -
      private Iterator<Cell> applySliceRestriction(final Iterator<Cell> cells, 
final QueryOptions options) throws InvalidRequestException
      {
 -        assert sliceRestriction != null;
 -
          final CellNameType type = cfm.comparator;
 +
          final CellName excludedStart = makeExclusiveSliceBound(Bound.START, 
type, options);
          final CellName excludedEnd = makeExclusiveSliceBound(Bound.END, type, 
options);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 91a059f,39e632a..517d842
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@@ -86,7 -84,7 +86,7 @@@ public class UpdateStatement extends Mo
          if (cfm.comparator.isDense())
          {
              if (prefix.isEmpty())
-                 throw new InvalidRequestException(String.format("Missing 
PRIMARY KEY part %s", cfm.clusteringColumns().get(0)));
 -                throw new InvalidRequestException(String.format("Missing 
PRIMARY KEY part %s", cfm.clusteringColumns().iterator().next().name));
++                throw new InvalidRequestException(String.format("Missing 
PRIMARY KEY part %s", cfm.clusteringColumns().get(0).name));
  
              // An empty name for the compact value is what we use to 
recognize the case where there is not column
              // outside the PK, see CreateStatement.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index d1fe702,2df3665..99ee62e
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -23,12 -23,12 +23,14 @@@ import java.util.*
  
  import com.google.common.annotations.VisibleForTesting;
  import com.google.common.base.Throwables;
+ import com.google.common.collect.AbstractIterator;
 -import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+ import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
@@@ -220,22 -222,11 +222,8 @@@ public class Scrubber implements Closea
                      if (indexFile != null && dataStart != dataStartFromIndex)
                          outputHandler.warn(String.format("Data file row 
position %d differs from index file row position %d", dataStart, 
dataStartFromIndex));
  
-                     SSTableIdentityIterator atoms = new 
SSTableIdentityIterator(sstable, dataFile, key, checkData);
- 
-                     if (prevKey != null && prevKey.compareTo(key) > 0)
-                     {
-                         saveOutOfOrderRow(prevKey, key, atoms);
-                         continue;
-                     }
- 
-                     @SuppressWarnings("resource")
-                     AbstractCompactedRow compactedRow = new 
LazilyCompactedRow(controller, Collections.singletonList(atoms));
-                     if (writer.tryAppend(compactedRow) == null)
-                         emptyRows++;
-                     else
-                         goodRows++;
 -                    if (indexFile != null && dataSize != dataSizeFromIndex)
 -                        outputHandler.warn(String.format("Data file row size 
%d different from index file row size %d", dataSize, dataSizeFromIndex));
--
-                     prevKey = key;
 -                    if (tryAppend(prevKey, key, dataSize, writer))
++                    if (tryAppend(prevKey, key, writer))
+                         prevKey = key;
                  }
                  catch (Throwable th)
                  {
@@@ -251,22 -242,8 +239,9 @@@
                          try
                          {
                              dataFile.seek(dataStartFromIndex);
 -                            if (tryAppend(prevKey, key, dataSize, writer))
 +
-                             SSTableIdentityIterator atoms = new 
SSTableIdentityIterator(sstable, dataFile, key, checkData);
-                             if (prevKey != null && prevKey.compareTo(key) > 0)
-                             {
-                                 saveOutOfOrderRow(prevKey, key, atoms);
-                                 continue;
-                             }
- 
-                             @SuppressWarnings("resource")
-                             AbstractCompactedRow compactedRow = new 
LazilyCompactedRow(controller, Collections.singletonList(atoms));
-                             if (writer.tryAppend(compactedRow) == null)
-                                 emptyRows++;
-                             else
-                                 goodRows++;
- 
-                             prevKey = key;
++                            if (tryAppend(prevKey, key, writer))
+                                 prevKey = key;
                          }
                          catch (Throwable th2)
                          {
@@@ -337,6 -316,32 +312,32 @@@
          }
      }
  
+     @SuppressWarnings("resource")
 -    private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, long 
dataSize, SSTableRewriter writer)
++    private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, 
SSTableRewriter writer)
+     {
+         // OrderCheckerIterator will check, at iteration time, that the cells 
are in the proper order. If it detects
+         // that one cell is out of order, it will stop returning them. The 
remaining cells will be sorted and added
+         // to the outOfOrderRows that will be later written to a new SSTable.
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(new 
SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
++        OrderCheckerIterator atoms = new OrderCheckerIterator(new 
SSTableIdentityIterator(sstable, dataFile, key, checkData),
+                                                               
cfs.metadata.comparator.onDiskAtomComparator());
+         if (prevKey != null && prevKey.compareTo(key) > 0)
+         {
+             saveOutOfOrderRow(prevKey, key, atoms);
+             return false;
+         }
+ 
+         AbstractCompactedRow compactedRow = new 
LazilyCompactedRow(controller, Collections.singletonList(atoms));
+         if (writer.tryAppend(compactedRow) == null)
+             emptyRows++;
+         else
+             goodRows++;
+ 
+         if (atoms.hasOutOfOrderCells())
+             saveOutOfOrderRow(key, atoms);
+ 
+         return true;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/db/marshal/ReversedType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/marshal/ReversedType.java
index 68a7e02,53798f8..19bee5f
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@@ -60,23 -58,8 +60,13 @@@ public class ReversedType<T> extends Ab
          this.baseType = baseType;
      }
  
 +    public boolean isEmptyValueMeaningless()
 +    {
 +        return baseType.isEmptyValueMeaningless();
 +    }
 +
      public int compare(ByteBuffer o1, ByteBuffer o2)
      {
-         // An empty byte buffer is always smaller
-         if (o1.remaining() == 0)
-         {
-             return o2.remaining() == 0 ? 0 : -1;
-         }
-         if (o2.remaining() == 0)
-         {
-             return 1;
-         }
- 
          return baseType.compare(o2, o1);
      }
  

Reply via email to