Skip to content

Commit d223b82

Browse files
committed
Merge branch 'develop' into feature/gt-3.0
* develop: (24 commits) remove unused imports Python unit tests check read pipeline stages Cruft removal. Fix for ML transformer read/write Add failing unit tests for issue 425 ML custom transformer loading is broken PR feedback. Fixed `ReprojectToLayer` to work with `proj_raster`. Closes #357 Depends on #420. break out commented assert into skipped unit test around masking and deserialization register rf_local_extract_bit with SQL functions Added the ability to do a raster_join on proj_raster types. Fixes #419. Add landsat masking section to masking docs page Add mask bits python api and unit test Extract bits should throw on non-integral cell types Fix for both masking by def and value; expand code comments; update tests Python regression. Masking improvements and unit tests. Regression Added (disabled) integration test for profiling spatial index effects. Reorganized non-evaluated documentationm files to `docs`. Add failing unit test for mask by value on 0 Updated python tests against spatial index functions. ... # Conflicts: # core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala # core/src/main/scala/org/locationtech/rasterframes/expressions/transformers/XZ2Indexer.scala # core/src/main/scala/org/locationtech/rasterframes/extensions/Implicits.scala # core/src/test/scala/examples/CreatingRasterFrames.scala # core/src/test/scala/org/locationtech/rasterframes/RasterLayerSpec.scala # core/src/test/scala/org/locationtech/rasterframes/expressions/XZ2IndexerSpec.scala # datasource/src/main/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceRelation.scala # datasource/src/test/scala/org/locationtech/rasterframes/datasource/raster/RasterSourceDataSourceSpec.scala
2 parents 7345251 + d8ea781 commit d223b82

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

46 files changed

+1548
-523
lines changed

build.sbt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ lazy val root = project
3434
.enablePlugins(RFReleasePlugin)
3535
.settings(
3636
publish / skip := true,
37-
clean := clean.dependsOn(`rf-notebook`/clean).value
37+
clean := clean.dependsOn(`rf-notebook`/clean, docs/clean).value
3838
)
3939

4040
lazy val `rf-notebook` = project

core/src/main/scala/org/locationtech/rasterframes/RasterFunctions.scala

Lines changed: 76 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -52,30 +52,50 @@ trait RasterFunctions {
5252
/** Query the number of (cols, rows) in a Tile. */
5353
def rf_dimensions(col: Column): TypedColumn[Any, Dimensions[Int]] = GetDimensions(col)
5454

55+
/** Extracts the CRS from a RasterSource or ProjectedRasterTile */
56+
def rf_crs(col: Column): TypedColumn[Any, CRS] = GetCRS(col)
57+
5558
/** Extracts the bounding box of a geometry as an Extent */
5659
def st_extent(col: Column): TypedColumn[Any, Extent] = GeometryToExtent(col)
5760

5861
/** Extracts the bounding box from a RasterSource or ProjectedRasterTile */
5962
def rf_extent(col: Column): TypedColumn[Any, Extent] = GetExtent(col)
6063

61-
/** Constructs a XZ2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS
64+
/** Constructs a XZ2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS.
6265
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
63-
def rf_spatial_index(targetExtent: Column, targetCRS: Column, indexResolution: Short) = XZ2Indexer(targetExtent, targetCRS, indexResolution)
66+
def rf_xz2_index(targetExtent: Column, targetCRS: Column, indexResolution: Short) = XZ2Indexer(targetExtent, targetCRS, indexResolution)
6467

6568
/** Constructs a XZ2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS
6669
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
67-
def rf_spatial_index(targetExtent: Column, targetCRS: Column) = XZ2Indexer(targetExtent, targetCRS, 18: Short)
70+
def rf_xz2_index(targetExtent: Column, targetCRS: Column) = XZ2Indexer(targetExtent, targetCRS, 18: Short)
6871

69-
/** Constructs a XZ2 index with level 18 resolution in WGS84 from either a ProjectedRasterTile or RasterSource
72+
/** Constructs a XZ2 index with provided resolution level in WGS84 from either a ProjectedRasterTile or RasterSource.
7073
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
71-
def rf_spatial_index(targetExtent: Column, indexResolution: Short) = XZ2Indexer(targetExtent, indexResolution)
74+
def rf_xz2_index(targetExtent: Column, indexResolution: Short) = XZ2Indexer(targetExtent, indexResolution)
7275

73-
/** Constructs a XZ2 index with level 18 resolution in WGS84 from either a ProjectedRasterTile or RasterSource
76+
/** Constructs a XZ2 index with level 18 resolution in WGS84 from either a ProjectedRasterTile or RasterSource.
7477
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
75-
def rf_spatial_index(targetExtent: Column) = XZ2Indexer(targetExtent, 18: Short)
78+
def rf_xz2_index(targetExtent: Column) = XZ2Indexer(targetExtent, 18: Short)
7679

77-
/** Extracts the CRS from a RasterSource or ProjectedRasterTile */
78-
def rf_crs(col: Column): TypedColumn[Any, CRS] = GetCRS(col)
80+
/** Constructs a Z2 index in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS.
81+
* First the native extent is extracted or computed, and then center is used as the indexing location.
82+
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
83+
def rf_z2_index(targetExtent: Column, targetCRS: Column, indexResolution: Short) = Z2Indexer(targetExtent, targetCRS, indexResolution)
84+
85+
/** Constructs a Z2 index with index resolution of 31 in WGS84 from either a Geometry, Extent, ProjectedRasterTile, or RasterSource and its CRS.
86+
* First the native extent is extracted or computed, and then center is used as the indexing location.
87+
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
88+
def rf_z2_index(targetExtent: Column, targetCRS: Column) = Z2Indexer(targetExtent, targetCRS, 31: Short)
89+
90+
/** Constructs a Z2 index with the given index resolution in WGS84 from either a ProjectedRasterTile or RasterSource
91+
* First the native extent is extracted or computed, and then center is used as the indexing location.
92+
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
93+
def rf_z2_index(targetExtent: Column, indexResolution: Short) = Z2Indexer(targetExtent, indexResolution)
94+
95+
/** Constructs a Z2 index with index resolution of 31 in WGS84 from either a ProjectedRasterTile or RasterSource
96+
* First the native extent is extracted or computed, and then center is used as the indexing location.
97+
* For details: https://www.geomesa.org/documentation/user/datastores/index_overview.html */
98+
def rf_z2_index(targetExtent: Column) = Z2Indexer(targetExtent, 31: Short)
7999

80100
/** Extracts the tile from a ProjectedRasterTile, or passes through a Tile. */
81101
def rf_tile(col: Column): TypedColumn[Any, Tile] = RealizeTile(col)
@@ -318,7 +338,7 @@ trait RasterFunctions {
318338

319339
/** Generate a tile with the values from `data_tile`, but where cells in the `mask_tile` are in the `mask_values`
320340
list, replace the value with NODATA. */
321-
def rf_mask_by_values(sourceTile: Column, maskTile: Column, maskValues: Seq[Int]): TypedColumn[Any, Tile] = {
341+
def rf_mask_by_values(sourceTile: Column, maskTile: Column, maskValues: Int*): TypedColumn[Any, Tile] = {
322342
import org.apache.spark.sql.functions.array
323343
val valuesCol: Column = array(maskValues.map(lit).toSeq: _*)
324344
rf_mask_by_values(sourceTile, maskTile, valuesCol)
@@ -336,6 +356,52 @@ trait RasterFunctions {
336356
def rf_inverse_mask_by_value(sourceTile: Column, maskTile: Column, maskValue: Int): TypedColumn[Any, Tile] =
337357
Mask.InverseMaskByValue(sourceTile, maskTile, lit(maskValue))
338358

359+
/** Applies a mask using bit values in the `mask_tile`. Working from the right, extract the bit at `bitPosition` from the `maskTile`. In all locations where these are equal to the `valueToMask`, the returned tile is set to NoData, else the original `dataTile` cell value. */
360+
def rf_mask_by_bit(dataTile: Column, maskTile: Column, bitPosition: Int, valueToMask: Boolean): TypedColumn[Any, Tile] =
361+
rf_mask_by_bit(dataTile, maskTile, lit(bitPosition), lit(if (valueToMask) 1 else 0))
362+
363+
/** Applies a mask using bit values in the `mask_tile`. Working from the right, extract the bit at `bitPosition` from the `maskTile`. In all locations where these are equal to the `valueToMask`, the returned tile is set to NoData, else the original `dataTile` cell value. */
364+
def rf_mask_by_bit(dataTile: Column, maskTile: Column, bitPosition: Column, valueToMask: Column): TypedColumn[Any, Tile] = {
365+
import org.apache.spark.sql.functions.array
366+
rf_mask_by_bits(dataTile, maskTile, bitPosition, lit(1), array(valueToMask))
367+
}
368+
369+
/** Applies a mask from blacklisted bit values in the `mask_tile`. Working from the right, the bits from `start_bit` to `start_bit + num_bits` are @ref:[extracted](reference.md#rf_local_extract_bits) from cell values of the `mask_tile`. In all locations where these are in the `mask_values`, the returned tile is set to NoData; otherwise the original `tile` cell value is returned. */
370+
def rf_mask_by_bits(dataTile: Column, maskTile: Column, startBit: Column, numBits: Column, valuesToMask: Column): TypedColumn[Any, Tile] = {
371+
val bitMask = rf_local_extract_bits(maskTile, startBit, numBits)
372+
rf_mask_by_values(dataTile, bitMask, valuesToMask)
373+
}
374+
375+
376+
/** Applies a mask from blacklisted bit values in the `mask_tile`. Working from the right, the bits from `start_bit` to `start_bit + num_bits` are @ref:[extracted](reference.md#rf_local_extract_bits) from cell values of the `mask_tile`. In all locations where these are in the `mask_values`, the returned tile is set to NoData; otherwise the original `tile` cell value is returned. */
377+
def rf_mask_by_bits(dataTile: Column, maskTile: Column, startBit: Int, numBits: Int, valuesToMask: Int*): TypedColumn[Any, Tile] = {
378+
import org.apache.spark.sql.functions.array
379+
val values = array(valuesToMask.map(lit):_*)
380+
rf_mask_by_bits(dataTile, maskTile, lit(startBit), lit(numBits), values)
381+
}
382+
383+
/** Extract value from specified bits of the cells' underlying binary data.
384+
* `startBit` is the first bit to consider, working from the right. It is zero indexed.
385+
* `numBits` is the number of bits to take moving further to the left. */
386+
def rf_local_extract_bits(tile: Column, startBit: Column, numBits: Column): Column =
387+
ExtractBits(tile, startBit, numBits)
388+
389+
/** Extract value from specified bits of the cells' underlying binary data.
390+
* `bitPosition` is bit to consider, working from the right. It is zero indexed. */
391+
def rf_local_extract_bits(tile: Column, bitPosition: Column): Column =
392+
rf_local_extract_bits(tile, bitPosition, lit(1))
393+
394+
/** Extract value from specified bits of the cells' underlying binary data.
395+
* `startBit` is the first bit to consider, working from the right. It is zero indexed.
396+
* `numBits` is the number of bits to take, moving further to the left. */
397+
def rf_local_extract_bits(tile: Column, startBit: Int, numBits: Int): Column =
398+
rf_local_extract_bits(tile, lit(startBit), lit(numBits))
399+
400+
/** Extract value from specified bits of the cells' underlying binary data.
401+
* `bitPosition` is bit to consider, working from the right. It is zero indexed. */
402+
def rf_local_extract_bits(tile: Column, bitPosition: Int): Column =
403+
rf_local_extract_bits(tile, lit(bitPosition))
404+
339405
/** Create a tile where cells in the grid defined by cols, rows, and bounds are filled with the given value. */
340406
def rf_rasterize(geometry: Column, bounds: Column, value: Column, cols: Int, rows: Int): TypedColumn[Any, Tile] =
341407
withTypedAlias("rf_rasterize", geometry)(

core/src/main/scala/org/locationtech/rasterframes/expressions/DynamicExtractors.scala

Lines changed: 36 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.sql.jts.JTSTypes
3030
import org.apache.spark.sql.rf.{RasterSourceUDT, TileUDT}
3131
import org.apache.spark.sql.types._
3232
import org.apache.spark.unsafe.types.UTF8String
33-
import org.locationtech.jts.geom.Envelope
33+
import org.locationtech.jts.geom.{Envelope, Point}
3434
import org.locationtech.rasterframes.encoders.CatalystSerializer._
3535
import org.locationtech.rasterframes.model.{LazyCRS, TileContext}
3636
import org.locationtech.rasterframes.ref.{ProjectedRasterLike, RasterRef, RFRasterSource}
@@ -69,13 +69,13 @@ object DynamicExtractors {
6969
}
7070

7171
/** Partial function for pulling a ProjectedRasterLike an input row. */
72-
lazy val projectedRasterLikeExtractor: PartialFunction[DataType, InternalRow ProjectedRasterLike] = {
72+
lazy val projectedRasterLikeExtractor: PartialFunction[DataType, Any ProjectedRasterLike] = {
7373
case _: RasterSourceUDT
74-
(row: InternalRow) => row.to[RFRasterSource](RasterSourceUDT.rasterSourceSerializer)
74+
(input: Any) => input.asInstanceOf[InternalRow].to[RFRasterSource](RasterSourceUDT.rasterSourceSerializer)
7575
case t if t.conformsTo[ProjectedRasterTile] =>
76-
(row: InternalRow) => row.to[ProjectedRasterTile]
76+
(input: Any) => input.asInstanceOf[InternalRow].to[ProjectedRasterTile]
7777
case t if t.conformsTo[RasterRef] =>
78-
(row: InternalRow) => row.to[RasterRef]
78+
(input: Any) => input.asInstanceOf[InternalRow].to[RasterRef]
7979
}
8080

8181
/** Partial function for pulling a CellGrid from an input row. */
@@ -97,13 +97,36 @@ object DynamicExtractors {
9797
(v: Any) => v.asInstanceOf[InternalRow].to[CRS]
9898
}
9999

100-
lazy val extentLikeExtractor: PartialFunction[DataType, Any Extent] = {
101-
case t if org.apache.spark.sql.rf.WithTypeConformity(t).conformsTo(JTSTypes.GeometryTypeInstance) =>
102-
(input: Any) => JTSTypes.GeometryTypeInstance.deserialize(input).getEnvelopeInternal
103-
case t if t.conformsTo[Extent] =>
104-
(input: Any) => input.asInstanceOf[InternalRow].to[Extent]
105-
case t if t.conformsTo[Envelope] =>
106-
(input: Any) => Extent(input.asInstanceOf[InternalRow].to[Envelope])
100+
lazy val extentExtractor: PartialFunction[DataType, Any Extent] = {
101+
val base: PartialFunction[DataType, Any Extent]= {
102+
case t if org.apache.spark.sql.rf.WithTypeConformity(t).conformsTo(JTSTypes.GeometryTypeInstance) =>
103+
(input: Any) => Extent(JTSTypes.GeometryTypeInstance.deserialize(input).getEnvelopeInternal)
104+
case t if t.conformsTo[Extent] =>
105+
(input: Any) => input.asInstanceOf[InternalRow].to[Extent]
106+
case t if t.conformsTo[Envelope] =>
107+
(input: Any) => Extent(input.asInstanceOf[InternalRow].to[Envelope])
108+
}
109+
110+
val fromPRL = projectedRasterLikeExtractor.andThen(_.andThen(_.extent))
111+
fromPRL orElse base
112+
}
113+
114+
lazy val envelopeExtractor: PartialFunction[DataType, Any => Envelope] = {
115+
val base = PartialFunction[DataType, Any => Envelope] {
116+
case t if org.apache.spark.sql.rf.WithTypeConformity(t).conformsTo(JTSTypes.GeometryTypeInstance) =>
117+
(input: Any) => JTSTypes.GeometryTypeInstance.deserialize(input).getEnvelopeInternal
118+
case t if t.conformsTo[Extent] =>
119+
(input: Any) => input.asInstanceOf[InternalRow].to[Extent].jtsEnvelope
120+
case t if t.conformsTo[Envelope] =>
121+
(input: Any) => input.asInstanceOf[InternalRow].to[Envelope]
122+
}
123+
124+
val fromPRL = projectedRasterLikeExtractor.andThen(_.andThen(_.extent.jtsEnvelope))
125+
fromPRL orElse base
126+
}
127+
128+
lazy val centroidExtractor: PartialFunction[DataType, Any Point] = {
129+
extentExtractor.andThen(_.andThen(_.center))
107130
}
108131

109132
sealed trait TileOrNumberArg
@@ -130,8 +153,7 @@ object DynamicExtractors {
130153
case _: DoubleType | _: FloatType | _: DecimalType => {
131154
case d: Double => DoubleArg(d)
132155
case f: Float => DoubleArg(f.toDouble)
133-
case d: Decimal => DoubleArg(d.toDouble)
134-
}
156+
case d: Decimal => DoubleArg(d.toDouble) }
135157
}
136158

137159
lazy val intArgExtractor: PartialFunction[DataType, Any => IntegerArg] = {

core/src/main/scala/org/locationtech/rasterframes/expressions/aggregates/TileRasterizerAggregate.scala

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ package org.locationtech.rasterframes.expressions.aggregates
2424
import geotrellis.proj4.CRS
2525
import geotrellis.raster.reproject.Reproject
2626
import geotrellis.raster.resample.ResampleMethod
27-
import geotrellis.raster.{ArrayTile, CellType, Dimensions, MultibandTile, ProjectedRaster, Raster, Tile}
27+
import geotrellis.raster.{ArrayTile, CellType, Dimensions, GeoAttrsError, MultibandTile, ProjectedRaster, Raster, Tile}
2828
import geotrellis.layer._
2929
import geotrellis.vector.Extent
3030
import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction}
@@ -99,10 +99,9 @@ object TileRasterizerAggregate {
9999

100100
def apply(tlm: TileLayerMetadata[_], sampler: ResampleMethod): ProjectedRasterDefinition = {
101101
// Try to determine the actual dimensions of our data coverage
102-
val actualSize = tlm.layout.toRasterExtent().gridBoundsFor(tlm.extent) // <--- Do we have the math right here?
103-
val cols = actualSize.width
104-
val rows = actualSize.height
105-
new ProjectedRasterDefinition(cols, rows, tlm.cellType, tlm.crs, tlm.extent, sampler)
102+
val Dimensions(cols, rows) = tlm.totalDimensions
103+
require(cols <= Int.MaxValue && rows <= Int.MaxValue, s"Can't construct a Raster of size $cols x $rows. (Too big!)")
104+
new ProjectedRasterDefinition(cols.toInt, rows.toInt, tlm.cellType, tlm.crs, tlm.extent, sampler)
106105
}
107106
}
108107

core/src/main/scala/org/locationtech/rasterframes/expressions/package.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -135,8 +135,12 @@ package object expressions {
135135
registry.registerExpression[RenderPNG.RenderCompositePNG]("rf_render_png")
136136
registry.registerExpression[RGBComposite]("rf_rgb_composite")
137137

138-
registry.registerExpression[XZ2Indexer]("rf_spatial_index")
138+
registry.registerExpression[XZ2Indexer]("rf_xz2_index")
139+
registry.registerExpression[Z2Indexer]("rf_z2_index")
139140

140141
registry.registerExpression[transformers.ReprojectGeometry]("st_reproject")
142+
143+
registry.registerExpression[ExtractBits]("rf_local_extract_bits")
144+
registry.registerExpression[ExtractBits]("rf_local_extract_bit")
141145
}
142146
}
Lines changed: 96 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,96 @@
1+
/*
2+
* This software is licensed under the Apache 2 license, quoted below.
3+
*
4+
* Copyright 2019 Astraea, Inc.
5+
*
6+
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
7+
* use this file except in compliance with the License. You may obtain a copy of
8+
* the License at
9+
*
10+
* [http://www.apache.org/licenses/LICENSE-2.0]
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
14+
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
15+
* License for the specific language governing permissions and limitations under
16+
* the License.
17+
*
18+
* SPDX-License-Identifier: Apache-2.0
19+
*
20+
*/
21+
22+
package org.locationtech.rasterframes.expressions.transformers
23+
24+
import geotrellis.raster.Tile
25+
import org.apache.spark.sql.Column
26+
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
27+
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess}
28+
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
29+
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, TernaryExpression}
30+
import org.apache.spark.sql.rf.TileUDT
31+
import org.apache.spark.sql.types.DataType
32+
import org.locationtech.rasterframes.encoders.CatalystSerializer._
33+
import org.locationtech.rasterframes.expressions.DynamicExtractors._
34+
import org.locationtech.rasterframes.expressions._
35+
36+
@ExpressionDescription(
37+
usage = "_FUNC_(tile, start_bit, num_bits) - In each cell of `tile`, extract `num_bits` from the cell value, starting at `start_bit` from the left.",
38+
arguments = """
39+
Arguments:
40+
* tile - tile column to extract values
41+
* start_bit -
42+
* num_bits -
43+
""",
44+
examples = """
45+
Examples:
46+
> SELECT _FUNC_(tile, lit(4), lit(2))
47+
..."""
48+
)
49+
case class ExtractBits(child1: Expression, child2: Expression, child3: Expression) extends TernaryExpression with CodegenFallback with Serializable {
50+
override val nodeName: String = "rf_local_extract_bits"
51+
52+
override def children: Seq[Expression] = Seq(child1, child2, child3)
53+
54+
override def dataType: DataType = child1.dataType
55+
56+
override def checkInputDataTypes(): TypeCheckResult =
57+
if(!tileExtractor.isDefinedAt(child1.dataType)) {
58+
TypeCheckFailure(s"Input type '${child1.dataType}' does not conform to a raster type.")
59+
} else if (!intArgExtractor.isDefinedAt(child2.dataType)) {
60+
TypeCheckFailure(s"Input type '${child2.dataType}' isn't an integral type.")
61+
} else if (!intArgExtractor.isDefinedAt(child3.dataType)) {
62+
TypeCheckFailure(s"Input type '${child3.dataType}' isn't an integral type.")
63+
} else TypeCheckSuccess
64+
65+
66+
override protected def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = {
67+
implicit val tileSer = TileUDT.tileSerializer
68+
val (childTile, childCtx) = tileExtractor(child1.dataType)(row(input1))
69+
70+
val startBits = intArgExtractor(child2.dataType)(input2).value
71+
72+
val numBits = intArgExtractor(child2.dataType)(input3).value
73+
74+
childCtx match {
75+
case Some(ctx) => ctx.toProjectRasterTile(op(childTile, startBits, numBits)).toInternalRow
76+
case None => op(childTile, startBits, numBits).toInternalRow
77+
}
78+
}
79+
80+
protected def op(tile: Tile, startBit: Int, numBits: Int): Tile = ExtractBits(tile, startBit, numBits)
81+
82+
}
83+
84+
object ExtractBits{
85+
def apply(tile: Column, startBit: Column, numBits: Column): Column =
86+
new Column(ExtractBits(tile.expr, startBit.expr, numBits.expr))
87+
88+
def apply(tile: Tile, startBit: Int, numBits: Int): Tile = {
89+
assert(!tile.cellType.isFloatingPoint, "ExtractBits operation requires integral CellType")
90+
// this is the last `numBits` positions of "111111111111111"
91+
val widthMask = Int.MaxValue >> (63 - numBits)
92+
// map preserving the nodata structure
93+
tile.mapIfSet(x x >> startBit & widthMask)
94+
}
95+
96+
}

0 commit comments

Comments
 (0)