mirror of https://github.com/apache/lucene.git
Merge branch 'master' into jira/solr-8593
This commit is contained in:
commit
aae6f2703f
|
@ -132,7 +132,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
// If we're right on top of any of the test planes, we navigate solely on that plane.
|
||||
if (testPointFixedYPlane.evaluateIsZero(thePoint)) {
|
||||
// Use the XZ plane exclusively.
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedYPlane, testPointFixedYAbovePlane, testPointFixedYBelowPlane, testPoint, thePoint);
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedYPlane, testPointFixedYAbovePlane, testPointFixedYBelowPlane, thePoint);
|
||||
// Traverse our way from the test point to the check point. Use the y tree because that's fixed.
|
||||
if (!yTree.traverse(crossingEdgeIterator, testPoint.y)) {
|
||||
// Endpoint is on edge
|
||||
|
@ -141,7 +141,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
return ((crossingEdgeIterator.crossingCount & 1) == 0)?testPointInSet:!testPointInSet;
|
||||
} else if (testPointFixedXPlane.evaluateIsZero(thePoint)) {
|
||||
// Use the YZ plane exclusively.
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedXPlane, testPointFixedXAbovePlane, testPointFixedXBelowPlane, testPoint, thePoint);
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedXPlane, testPointFixedXAbovePlane, testPointFixedXBelowPlane, thePoint);
|
||||
// Traverse our way from the test point to the check point. Use the x tree because that's fixed.
|
||||
if (!xTree.traverse(crossingEdgeIterator, testPoint.x)) {
|
||||
// Endpoint is on edge
|
||||
|
@ -150,7 +150,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
return ((crossingEdgeIterator.crossingCount & 1) == 0)?testPointInSet:!testPointInSet;
|
||||
} else if (testPointFixedZPlane.evaluateIsZero(thePoint)) {
|
||||
// Use the XY plane exclusively.
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedZPlane, testPointFixedZAbovePlane, testPointFixedZBelowPlane, testPoint, thePoint);
|
||||
final LinearCrossingEdgeIterator crossingEdgeIterator = new LinearCrossingEdgeIterator(testPointFixedZPlane, testPointFixedZAbovePlane, testPointFixedZBelowPlane, thePoint);
|
||||
// Traverse our way from the test point to the check point. Use the z tree because that's fixed.
|
||||
if (!zTree.traverse(crossingEdgeIterator, testPoint.z)) {
|
||||
// Endpoint is on edge
|
||||
|
@ -159,6 +159,9 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
return ((crossingEdgeIterator.crossingCount & 1) == 0)?testPointInSet:!testPointInSet;
|
||||
} else {
|
||||
|
||||
// This is the expensive part!!
|
||||
// Changing the code below has an enormous impact on the queries per second we see with the benchmark.
|
||||
|
||||
// We need to use two planes to get there. We don't know which two planes will do it but we can figure it out.
|
||||
final Plane travelPlaneFixedX = new Plane(1.0, 0.0, 0.0, -thePoint.x);
|
||||
final Plane travelPlaneFixedY = new Plane(0.0, 1.0, 0.0, -thePoint.y);
|
||||
|
@ -186,7 +189,15 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
|
||||
for (final GeoPoint p : XIntersectionsY) {
|
||||
// Travel would be in YZ plane (fixed x) then in XZ (fixed y)
|
||||
final double newDistance = Math.abs(testPoint.x - p.x) + Math.abs(thePoint.y - p.y);
|
||||
// We compute distance we need to travel as a placeholder for the number of intersections we might encounter.
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.x - p.x;
|
||||
final double tpDelta2 = testPoint.z - p.z;
|
||||
final double cpDelta1 = thePoint.y - p.y;
|
||||
final double cpDelta2 = thePoint.z - p.z;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.x - p.x) * (testPoint.x - p.x) + (testPoint.z - p.z) * (testPoint.z - p.z) + (thePoint.y - p.y) * (thePoint.y - p.y) + (thePoint.z - p.z) * (thePoint.z - p.z);
|
||||
//final double newDistance = Math.abs(testPoint.x - p.x) + Math.abs(thePoint.y - p.y);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.y;
|
||||
|
@ -202,7 +213,14 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
}
|
||||
for (final GeoPoint p : XIntersectionsZ) {
|
||||
// Travel would be in YZ plane (fixed x) then in XY (fixed z)
|
||||
final double newDistance = Math.abs(testPoint.x - p.x) + Math.abs(thePoint.z - p.z);
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.x - p.x;
|
||||
final double tpDelta2 = testPoint.y - p.y;
|
||||
final double cpDelta1 = thePoint.y - p.y;
|
||||
final double cpDelta2 = thePoint.z - p.z;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.x - p.x) * (testPoint.x - p.x) + (testPoint.y - p.y) * (testPoint.y - p.y) + (thePoint.y - p.y) * (thePoint.y - p.y) + (thePoint.z - p.z) * (thePoint.z - p.z);
|
||||
//final double newDistance = Math.abs(testPoint.x - p.x) + Math.abs(thePoint.z - p.z);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.z;
|
||||
|
@ -218,7 +236,14 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
}
|
||||
for (final GeoPoint p : YIntersectionsX) {
|
||||
// Travel would be in XZ plane (fixed y) then in YZ (fixed x)
|
||||
final double newDistance = Math.abs(testPoint.y - p.y) + Math.abs(thePoint.x - p.x);
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.y - p.y;
|
||||
final double tpDelta2 = testPoint.z - p.z;
|
||||
final double cpDelta1 = thePoint.x - p.x;
|
||||
final double cpDelta2 = thePoint.z - p.z;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.y - p.y) * (testPoint.y - p.y) + (testPoint.z - p.z) * (testPoint.z - p.z) + (thePoint.x - p.x) * (thePoint.x - p.x) + (thePoint.z - p.z) * (thePoint.z - p.z);
|
||||
//final double newDistance = Math.abs(testPoint.y - p.y) + Math.abs(thePoint.x - p.x);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.x;
|
||||
|
@ -234,7 +259,14 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
}
|
||||
for (final GeoPoint p : YIntersectionsZ) {
|
||||
// Travel would be in XZ plane (fixed y) then in XY (fixed z)
|
||||
final double newDistance = Math.abs(testPoint.y - p.y) + Math.abs(thePoint.z - p.z);
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.x - p.x;
|
||||
final double tpDelta2 = testPoint.y - p.y;
|
||||
final double cpDelta1 = thePoint.x - p.x;
|
||||
final double cpDelta2 = thePoint.z - p.z;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.x - p.x) * (testPoint.x - p.x) + (testPoint.y - p.y) * (testPoint.y - p.y) + (thePoint.x - p.x) * (thePoint.x - p.x) + (thePoint.z - p.z) * (thePoint.z - p.z);
|
||||
//final double newDistance = Math.abs(testPoint.y - p.y) + Math.abs(thePoint.z - p.z);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.z;
|
||||
|
@ -250,7 +282,14 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
}
|
||||
for (final GeoPoint p : ZIntersectionsX) {
|
||||
// Travel would be in XY plane (fixed z) then in YZ (fixed x)
|
||||
final double newDistance = Math.abs(testPoint.z - p.z) + Math.abs(thePoint.x - p.x);
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.y - p.y;
|
||||
final double tpDelta2 = testPoint.z - p.z;
|
||||
final double cpDelta1 = thePoint.y - p.y;
|
||||
final double cpDelta2 = thePoint.x - p.x;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.y - p.y) * (testPoint.y - p.y) + (testPoint.z - p.z) * (testPoint.z - p.z) + (thePoint.y - p.y) * (thePoint.y - p.y) + (thePoint.x - p.x) * (thePoint.x - p.x);
|
||||
//final double newDistance = Math.abs(testPoint.z - p.z) + Math.abs(thePoint.x - p.x);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.x;
|
||||
|
@ -266,7 +305,14 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
}
|
||||
for (final GeoPoint p : ZIntersectionsY) {
|
||||
// Travel would be in XY plane (fixed z) then in XZ (fixed y)
|
||||
final double newDistance = Math.abs(testPoint.z - p.z) + Math.abs(thePoint.y - p.y);
|
||||
//final double newDistance = p.arcDistance(testPoint) + p.arcDistance(thePoint);
|
||||
final double tpDelta1 = testPoint.x - p.x;
|
||||
final double tpDelta2 = testPoint.z - p.z;
|
||||
final double cpDelta1 = thePoint.y - p.y;
|
||||
final double cpDelta2 = thePoint.x - p.x;
|
||||
final double newDistance = tpDelta1 * tpDelta1 + tpDelta2 * tpDelta2 + cpDelta1 * cpDelta1 + cpDelta2 * cpDelta2;
|
||||
//final double newDistance = (testPoint.x - p.x) * (testPoint.x - p.x) + (testPoint.z - p.z) * (testPoint.z - p.z) + (thePoint.y - p.y) * (thePoint.y - p.y) + (thePoint.x - p.x) * (thePoint.x - p.x);
|
||||
//final double newDistance = Math.abs(testPoint.z - p.z) + Math.abs(thePoint.y - p.y);
|
||||
if (newDistance < bestDistance) {
|
||||
bestDistance = newDistance;
|
||||
firstLegValue = testPoint.y;
|
||||
|
@ -284,7 +330,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
assert bestDistance > 0.0 : "Best distance should not be zero unless on single plane";
|
||||
assert bestDistance < Double.MAX_VALUE : "Couldn't find an intersection point of any kind";
|
||||
|
||||
final DualCrossingEdgeIterator edgeIterator = new DualCrossingEdgeIterator(firstLegPlane, firstLegAbovePlane, firstLegBelowPlane, secondLegPlane, testPoint, thePoint, intersectionPoint);
|
||||
final DualCrossingEdgeIterator edgeIterator = new DualCrossingEdgeIterator(firstLegPlane, firstLegAbovePlane, firstLegBelowPlane, secondLegPlane, thePoint, intersectionPoint);
|
||||
if (!firstLegTree.traverse(edgeIterator, firstLegValue)) {
|
||||
return true;
|
||||
}
|
||||
|
@ -666,7 +712,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
|
||||
public int crossingCount = 0;
|
||||
|
||||
public LinearCrossingEdgeIterator(final Plane plane, final Plane abovePlane, final Plane belowPlane, final Vector testPoint, final Vector thePoint) {
|
||||
public LinearCrossingEdgeIterator(final Plane plane, final Plane abovePlane, final Plane belowPlane, final Vector thePoint) {
|
||||
this.plane = plane;
|
||||
this.abovePlane = abovePlane;
|
||||
this.belowPlane = belowPlane;
|
||||
|
@ -815,11 +861,9 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
private boolean isSecondLeg = false;
|
||||
|
||||
private final Plane testPointPlane;
|
||||
private final Plane testPointInsidePlane;
|
||||
private final Plane testPointOutsidePlane;
|
||||
private final Plane testPointAbovePlane;
|
||||
private final Plane testPointBelowPlane;
|
||||
private final Plane travelPlane;
|
||||
private final Plane travelInsidePlane;
|
||||
private final Plane travelOutsidePlane;
|
||||
private final Vector thePoint;
|
||||
|
||||
private final GeoPoint intersectionPoint;
|
||||
|
@ -829,14 +873,25 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
private final SidedPlane testPointOtherCutoffPlane;
|
||||
private final SidedPlane checkPointOtherCutoffPlane;
|
||||
|
||||
private final SidedPlane insideTestPointCutoffPlane;
|
||||
private final SidedPlane insideTravelCutoffPlane;
|
||||
// These are computed on an as-needed basis
|
||||
|
||||
private boolean computedInsideOutside = false;
|
||||
private Plane testPointInsidePlane;
|
||||
private Plane testPointOutsidePlane;
|
||||
private Plane travelInsidePlane;
|
||||
private Plane travelOutsidePlane;
|
||||
private SidedPlane insideTestPointCutoffPlane;
|
||||
private SidedPlane insideTravelCutoffPlane;
|
||||
|
||||
// The counter
|
||||
|
||||
public int crossingCount = 0;
|
||||
|
||||
public DualCrossingEdgeIterator(final Plane testPointPlane, final Plane testPointAbovePlane, final Plane testPointBelowPlane,
|
||||
final Plane travelPlane, final Vector testPoint, final Vector thePoint, final GeoPoint intersectionPoint) {
|
||||
final Plane travelPlane, final Vector thePoint, final GeoPoint intersectionPoint) {
|
||||
this.testPointPlane = testPointPlane;
|
||||
this.testPointAbovePlane = testPointAbovePlane;
|
||||
this.testPointBelowPlane = testPointBelowPlane;
|
||||
this.travelPlane = travelPlane;
|
||||
this.thePoint = thePoint;
|
||||
this.intersectionPoint = intersectionPoint;
|
||||
|
@ -854,16 +909,21 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
this.testPointOtherCutoffPlane = new SidedPlane(testPoint, testPointPlane, intersectionPoint);
|
||||
this.checkPointOtherCutoffPlane = new SidedPlane(thePoint, travelPlane, intersectionPoint);
|
||||
|
||||
// Convert travel plane to a sided plane
|
||||
final Membership intersectionBound1 = new SidedPlane(testPoint, travelPlane, travelPlane.D);
|
||||
// Convert testPoint plane to a sided plane
|
||||
final Membership intersectionBound2 = new SidedPlane(thePoint, testPointPlane, testPointPlane.D);
|
||||
|
||||
// Sanity check
|
||||
assert testPointCutoffPlane.isWithin(intersectionPoint) : "intersection must be within testPointCutoffPlane";
|
||||
assert testPointOtherCutoffPlane.isWithin(intersectionPoint) : "intersection must be within testPointOtherCutoffPlane";
|
||||
assert checkPointCutoffPlane.isWithin(intersectionPoint) : "intersection must be within checkPointCutoffPlane";
|
||||
assert checkPointOtherCutoffPlane.isWithin(intersectionPoint) : "intersection must be within checkPointOtherCutoffPlane";
|
||||
|
||||
}
|
||||
|
||||
protected void computeInsideOutside() {
|
||||
if (!computedInsideOutside) {
|
||||
// Convert travel plane to a sided plane
|
||||
final Membership intersectionBound1 = new SidedPlane(testPoint, travelPlane, travelPlane.D);
|
||||
// Convert testPoint plane to a sided plane
|
||||
final Membership intersectionBound2 = new SidedPlane(thePoint, testPointPlane, testPointPlane.D);
|
||||
|
||||
assert intersectionBound1.isWithin(intersectionPoint) : "intersection must be within intersectionBound1";
|
||||
assert intersectionBound2.isWithin(intersectionPoint) : "intersection must be within intersectionBound2";
|
||||
|
||||
|
@ -884,36 +944,32 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
|
||||
assert ((aboveAbove.length > 0)?1:0) + ((aboveBelow.length > 0)?1:0) + ((belowBelow.length > 0)?1:0) + ((belowAbove.length > 0)?1:0) == 1 : "Can be exactly one inside point, instead was: aa="+aboveAbove.length+" ab=" + aboveBelow.length+" bb="+ belowBelow.length+" ba=" + belowAbove.length;
|
||||
|
||||
final GeoPoint insideIntersection;
|
||||
if (aboveAbove.length > 0) {
|
||||
travelInsidePlane = travelAbovePlane;
|
||||
testPointInsidePlane = testPointAbovePlane;
|
||||
travelOutsidePlane = travelBelowPlane;
|
||||
testPointOutsidePlane = testPointBelowPlane;
|
||||
insideIntersection = aboveAbove[0];
|
||||
} else if (aboveBelow.length > 0) {
|
||||
travelInsidePlane = travelAbovePlane;
|
||||
testPointInsidePlane = testPointBelowPlane;
|
||||
travelOutsidePlane = travelBelowPlane;
|
||||
testPointOutsidePlane = testPointAbovePlane;
|
||||
insideIntersection = aboveBelow[0];
|
||||
} else if (belowBelow.length > 0) {
|
||||
travelInsidePlane = travelBelowPlane;
|
||||
testPointInsidePlane = testPointBelowPlane;
|
||||
travelOutsidePlane = travelAbovePlane;
|
||||
testPointOutsidePlane = testPointAbovePlane;
|
||||
insideIntersection = belowBelow[0];
|
||||
} else {
|
||||
travelInsidePlane = travelBelowPlane;
|
||||
testPointInsidePlane = testPointAbovePlane;
|
||||
travelOutsidePlane = travelAbovePlane;
|
||||
testPointOutsidePlane = testPointBelowPlane;
|
||||
insideIntersection = belowAbove[0];
|
||||
}
|
||||
|
||||
insideTravelCutoffPlane = new SidedPlane(thePoint, testPointInsidePlane, testPointInsidePlane.D);
|
||||
insideTestPointCutoffPlane = new SidedPlane(testPoint, travelInsidePlane, travelInsidePlane.D);
|
||||
|
||||
computedInsideOutside = true;
|
||||
}
|
||||
}
|
||||
|
||||
public void setSecondLeg() {
|
||||
|
@ -982,28 +1038,10 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
|
||||
// Plane crossing, either first leg or second leg
|
||||
|
||||
final Plane plane;
|
||||
final Plane insidePlane;
|
||||
final Plane outsidePlane;
|
||||
final SidedPlane bound1;
|
||||
final SidedPlane bound2;
|
||||
if (isSecondLeg) {
|
||||
plane = travelPlane;
|
||||
insidePlane = travelInsidePlane;
|
||||
outsidePlane = travelOutsidePlane;
|
||||
bound1 = checkPointCutoffPlane;
|
||||
bound2 = checkPointOtherCutoffPlane;
|
||||
} else {
|
||||
plane = testPointPlane;
|
||||
insidePlane = testPointInsidePlane;
|
||||
outsidePlane = testPointOutsidePlane;
|
||||
bound1 = testPointCutoffPlane;
|
||||
bound2 = testPointOtherCutoffPlane;
|
||||
}
|
||||
|
||||
if (crossingPoint.isNumericallyIdentical(edge.startPoint)) {
|
||||
//System.err.println(" Crossing point = edge.startPoint");
|
||||
// We have to figure out if this crossing should be counted.
|
||||
computeInsideOutside();
|
||||
|
||||
// Does the crossing for this edge go up, or down? Or can't we tell?
|
||||
final GeoPoint[] insideTestPointPlaneIntersections = testPointInsidePlane.findIntersections(planetModel, edge.plane, edge.startPlane, edge.endPlane, insideTestPointCutoffPlane);
|
||||
|
@ -1050,7 +1088,12 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
// a decision whether to count or not based on that.
|
||||
|
||||
// Compute the crossing points of this other edge.
|
||||
final GeoPoint[] otherCrossingPoints = plane.findCrossings(planetModel, assessEdge.plane, bound1, bound2, assessEdge.startPlane, assessEdge.endPlane);
|
||||
final GeoPoint[] otherCrossingPoints;
|
||||
if (isSecondLeg) {
|
||||
otherCrossingPoints = travelPlane.findCrossings(planetModel, assessEdge.plane, checkPointCutoffPlane, checkPointOtherCutoffPlane, assessEdge.startPlane, assessEdge.endPlane);
|
||||
} else {
|
||||
otherCrossingPoints = testPointPlane.findCrossings(planetModel, assessEdge.plane, testPointCutoffPlane, testPointOtherCutoffPlane, assessEdge.startPlane, assessEdge.endPlane);
|
||||
}
|
||||
|
||||
// Look for a matching endpoint. If the other endpoint doesn't show up, it is either out of bounds (in which case the
|
||||
// transition won't be counted for that edge), or it is not a crossing for that edge (so, same conclusion).
|
||||
|
@ -1079,6 +1122,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
|
|||
} else if (crossingPoint.isNumericallyIdentical(edge.endPoint)) {
|
||||
//System.err.println(" Crossing point = edge.endPoint");
|
||||
// Figure out if the crossing should be counted.
|
||||
computeInsideOutside();
|
||||
|
||||
// Does the crossing for this edge go up, or down? Or can't we tell?
|
||||
final GeoPoint[] insideTestPointPlaneIntersections = testPointInsidePlane.findIntersections(planetModel, edge.plane, edge.startPlane, edge.endPlane, insideTestPointCutoffPlane);
|
||||
|
|
|
@ -121,7 +121,7 @@ public class GeoPoint extends Vector {
|
|||
* @param v is the second point.
|
||||
* @return the angle, in radians, between the two points.
|
||||
*/
|
||||
public double arcDistance(final GeoPoint v) {
|
||||
public double arcDistance(final Vector v) {
|
||||
return Tools.safeAcos(dotProduct(v)/(magnitude() * v.magnitude()));
|
||||
}
|
||||
|
||||
|
|
|
@ -242,7 +242,7 @@ public class PlanetModel {
|
|||
* @param pt1 is the first point.
|
||||
* @param pt2 is the second point.
|
||||
* @return the adjusted angle, when multiplied by the mean earth radius, yields a surface distance. This will differ
|
||||
* from GeoPoint.arcDistance() only when the planet model is not a sphere. @see {@link GeoPoint#arcDistance(GeoPoint)}
|
||||
* from GeoPoint.arcDistance() only when the planet model is not a sphere. @see {@link GeoPoint#arcDistance(Vector)}
|
||||
*/
|
||||
public double surfaceDistance(final GeoPoint pt1, final GeoPoint pt2) {
|
||||
final double L = pt2.getLongitude() - pt1.getLongitude();
|
||||
|
|
|
@ -179,6 +179,11 @@ Bug Fixes
|
|||
|
||||
* SOLR-8792: ZooKeeper ACL support fixed. (Esther Quansah, Ishan Chattopadhyaya, Steve Rowe)
|
||||
|
||||
* SOLR-9064: Adds an explanation of the incoming stream to an UpdateStream's explanation (Dennis Gove)
|
||||
|
||||
* SOLR-9030: The 'downnode' overseer command can trip asserts in ZkStateWriter.
|
||||
(Scott Blum, Mark Miller, shalin)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
* SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.
|
||||
|
@ -241,7 +246,7 @@ Other Changes
|
|||
|
||||
* SOLR-9047: zkcli should allow alternative locations for log4j configuration (Gregory Chanan)
|
||||
|
||||
* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, janhoy)
|
||||
* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, Mike Drob via janhoy)
|
||||
|
||||
* SOLR-9066 Make CountMetric return long instead of double (Kevin Risden)
|
||||
|
||||
|
|
|
@ -233,8 +233,9 @@ public class Assign {
|
|||
}
|
||||
DocCollection coll = clusterState.getCollection(collectionName);
|
||||
Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
|
||||
for (String s : clusterState.getCollections()) {
|
||||
DocCollection c = clusterState.getCollection(s);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection c = entry.getValue();
|
||||
//identify suitable nodes by checking the no:of cores in each of them
|
||||
for (Slice slice : c.getSlices()) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
@ -242,7 +243,7 @@ public class Assign {
|
|||
ReplicaCount count = nodeNameVsShardCount.get(replica.getNodeName());
|
||||
if (count != null) {
|
||||
count.totalNodes++; // Used ot "weigh" whether this node should be used later.
|
||||
if (s.equals(collectionName)) {
|
||||
if (entry.getKey().equals(collectionName)) {
|
||||
count.thisCollectionNodes++;
|
||||
if (count.thisCollectionNodes >= maxShardsPerNode) nodeNameVsShardCount.remove(replica.getNodeName());
|
||||
}
|
||||
|
|
|
@ -162,10 +162,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
liveNodes = clusterState.getLiveNodes();
|
||||
lastClusterStateVersion = clusterState.getZkClusterStateVersion();
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (final String collection : collections) {
|
||||
log.debug("look at collection={}", collection);
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
log.debug("look at collection={}", entry.getKey());
|
||||
DocCollection docCollection = entry.getValue();
|
||||
if (!docCollection.getAutoAddReplicas()) {
|
||||
log.debug("Collection {} is not setup to use autoAddReplicas, skipping..", docCollection.getName());
|
||||
continue;
|
||||
|
@ -174,7 +174,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
log.debug("Skipping collection because it has no defined replicationFactor, name={}", docCollection.getName());
|
||||
continue;
|
||||
}
|
||||
log.debug("Found collection, name={} replicationFactor={}", collection, docCollection.getReplicationFactor());
|
||||
log.debug("Found collection, name={} replicationFactor={}", entry.getKey(), docCollection.getReplicationFactor());
|
||||
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
|
@ -188,7 +188,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
if (downReplicas.size() > 0 && goodReplicas < docCollection.getReplicationFactor()) {
|
||||
// badReplicaMap.put(collection, badReplicas);
|
||||
processBadReplicas(collection, downReplicas);
|
||||
processBadReplicas(entry.getKey(), downReplicas);
|
||||
} else if (goodReplicas > docCollection.getReplicationFactor()) {
|
||||
log.debug("There are too many replicas");
|
||||
}
|
||||
|
@ -313,10 +313,11 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (String collection : collections) {
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
String collection = entry.getKey();
|
||||
log.debug("look at collection {} as possible create candidate", collection);
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
DocCollection docCollection = entry.getValue();
|
||||
// TODO - only operate on collections with sharedfs failover = true ??
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
|
|
|
@ -350,10 +350,10 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
|
|||
throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet does not exist to delete: " + configSetName);
|
||||
}
|
||||
|
||||
for (String s : zkStateReader.getClusterState().getCollections()) {
|
||||
if (configSetName.equals(zkStateReader.readConfigName(s)))
|
||||
for (Map.Entry<String, DocCollection> entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) {
|
||||
if (configSetName.equals(zkStateReader.readConfigName(entry.getKey())))
|
||||
throw new SolrException(ErrorCode.BAD_REQUEST,
|
||||
"Can not delete ConfigSet as it is currently being used by collection [" + s + "]");
|
||||
"Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
|
||||
}
|
||||
|
||||
String propertyPath = ConfigSetProperties.DEFAULT_FILENAME;
|
||||
|
|
|
@ -689,14 +689,12 @@ public final class ZkController {
|
|||
long now = System.nanoTime();
|
||||
long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
boolean foundStates = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
|
||||
while (System.nanoTime() < timeout) {
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
collections = clusterState.getCollections();
|
||||
for (String collectionName : collections) {
|
||||
DocCollection collection = clusterState.getCollection(collectionName);
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection collection = entry.getValue();
|
||||
Collection<Slice> slices = collection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
|
|
@ -49,14 +49,13 @@ public class NodeMutator {
|
|||
|
||||
log.info("DownNode state invoked for node: " + nodeName);
|
||||
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (String collection : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Map<String,Slice> slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap());
|
||||
|
||||
Set<Entry<String,Slice>> entries = slicesCopy.entrySet();
|
||||
for (Entry<String,Slice> entry : entries) {
|
||||
Slice slice = docCollection.getSlice(entry.getKey());
|
||||
for (Entry<String,Slice> sliceEntry : slicesCopy.entrySet()) {
|
||||
Slice slice = docCollection.getSlice(sliceEntry.getKey());
|
||||
Map<String,Replica> newReplicas = new HashMap<String,Replica>();
|
||||
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
@ -77,7 +76,7 @@ public class NodeMutator {
|
|||
|
||||
}
|
||||
|
||||
zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy)));
|
||||
zkWriteCommands.add(new ZkWriteCommand(entry.getKey(), docCollection.copyWithSlices(slicesCopy)));
|
||||
}
|
||||
|
||||
return zkWriteCommands;
|
||||
|
|
|
@ -229,7 +229,6 @@ public class ZkStateWriter {
|
|||
byte[] data = Utils.toJSON(singletonMap(c.getName(), c));
|
||||
if (reader.getZkClient().exists(path, true)) {
|
||||
log.info("going to update_collection {} version: {}", path, c.getZNodeVersion());
|
||||
assert c.getZNodeVersion() >= 0;
|
||||
Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true);
|
||||
DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path);
|
||||
clusterState = clusterState.copyWith(name, newCollection);
|
||||
|
@ -251,13 +250,9 @@ public class ZkStateWriter {
|
|||
assert clusterState.getZkClusterStateVersion() >= 0;
|
||||
byte[] data = Utils.toJSON(clusterState);
|
||||
Stat stat = reader.getZkClient().setData(ZkStateReader.CLUSTER_STATE, data, clusterState.getZkClusterStateVersion(), true);
|
||||
Set<String> collectionNames = clusterState.getCollections();
|
||||
Map<String, DocCollection> collectionStates = new HashMap<>(collectionNames.size());
|
||||
for (String c : collectionNames) {
|
||||
collectionStates.put(c, clusterState.getCollection(c));
|
||||
}
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
// use the reader's live nodes because our cluster state's live nodes may be stale
|
||||
clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collectionStates);
|
||||
clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collections);
|
||||
isClusterStateModified = false;
|
||||
}
|
||||
lastUpdatedTime = System.nanoTime();
|
||||
|
|
|
@ -104,8 +104,9 @@ public class ReplicaAssigner {
|
|||
validateTags(nodeVsTags);
|
||||
|
||||
if (clusterState != null) {
|
||||
for (String s : clusterState.getCollections()) {
|
||||
DocCollection coll = clusterState.getCollection(s);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection coll = entry.getValue();
|
||||
for (Slice slice : coll.getSlices()) {
|
||||
for (Replica replica : slice.getReplicas()) {
|
||||
AtomicInteger count = nodeVsCores.get(replica.getNodeName());
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
|
|||
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.sql.CalciteSolrDriver;
|
||||
|
@ -252,7 +251,6 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
}
|
||||
|
||||
public void setStreamContext(StreamContext context) {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -89,20 +89,22 @@ public class ClusterStatus {
|
|||
byte[] bytes = Utils.toJSON(clusterState);
|
||||
Map<String, Object> stateMap = (Map<String,Object>) Utils.fromJSON(bytes);
|
||||
|
||||
Set<String> collections;
|
||||
String routeKey = message.getStr(ShardParams._ROUTE_);
|
||||
String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
|
||||
|
||||
Map<String, DocCollection> collectionsMap = null;
|
||||
if (collection == null) {
|
||||
collections = new HashSet<>(clusterState.getCollections());
|
||||
collectionsMap = clusterState.getCollectionsMap();
|
||||
} else {
|
||||
collections = Collections.singleton(collection);
|
||||
collectionsMap = Collections.singletonMap(collection, clusterState.getCollectionOrNull(collection));
|
||||
}
|
||||
|
||||
NamedList<Object> collectionProps = new SimpleOrderedMap<>();
|
||||
|
||||
for (String name : collections) {
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
Map<String, Object> collectionStatus;
|
||||
DocCollection clusterStateCollection = clusterState.getCollectionOrNull(name);
|
||||
String name = entry.getKey();
|
||||
DocCollection clusterStateCollection = entry.getValue();
|
||||
if (clusterStateCollection == null) {
|
||||
if (collection != null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
|
||||
|
|
|
@ -679,11 +679,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
|
|||
@Override
|
||||
Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler handler) throws Exception {
|
||||
NamedList<Object> results = new NamedList<>();
|
||||
Set<String> collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollections();
|
||||
List<String> collectionList = new ArrayList<>();
|
||||
for (String collection : collections) {
|
||||
collectionList.add(collection);
|
||||
}
|
||||
Map<String, DocCollection> collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollectionsMap();
|
||||
List<String> collectionList = new ArrayList<>(collections.keySet());
|
||||
results.add("collections", collectionList);
|
||||
SolrResponse response = new OverseerSolrResponse(results);
|
||||
rsp.getValues().addAll(response.getResponse());
|
||||
|
|
|
@ -815,15 +815,15 @@ public class HttpSolrCall {
|
|||
private void getSlicesForCollections(ClusterState clusterState,
|
||||
Collection<Slice> slices, boolean activeSlices) {
|
||||
if (activeSlices) {
|
||||
for (String collection : clusterState.getCollections()) {
|
||||
final Collection<Slice> activeCollectionSlices = clusterState.getActiveSlices(collection);
|
||||
for (Map.Entry<String, DocCollection> entry : clusterState.getCollectionsMap().entrySet()) {
|
||||
final Collection<Slice> activeCollectionSlices = entry.getValue().getActiveSlices();
|
||||
if (activeCollectionSlices != null) {
|
||||
slices.addAll(activeCollectionSlices);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (String collection : clusterState.getCollections()) {
|
||||
final Collection<Slice> collectionSlices = clusterState.getSlices(collection);
|
||||
for (Map.Entry<String, DocCollection> entry : clusterState.getCollectionsMap().entrySet()) {
|
||||
final Collection<Slice> collectionSlices = entry.getValue().getSlices();
|
||||
if (collectionSlices != null) {
|
||||
slices.addAll(collectionSlices);
|
||||
}
|
||||
|
|
|
@ -57,7 +57,6 @@ import org.apache.solr.core.SolrConfig;
|
|||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrQueryRequestBase;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.apache.solr.util.RTimerTree;
|
||||
|
||||
import static org.apache.solr.common.params.CommonParams.PATH;
|
||||
|
@ -564,11 +563,8 @@ public class SolrRequestParsers
|
|||
upload.setSizeMax( ((long) uploadLimitKB) * 1024L );
|
||||
|
||||
// Parse the request
|
||||
List items = upload.parseRequest(req);
|
||||
Iterator iter = items.iterator();
|
||||
while (iter.hasNext()) {
|
||||
FileItem item = (FileItem) iter.next();
|
||||
|
||||
List<FileItem> items = upload.parseRequest(req);
|
||||
for (FileItem item : items) {
|
||||
// If it's a form field, put it in our parameter map
|
||||
if (item.isFormField()) {
|
||||
MultiMapSolrParams.addParam(
|
||||
|
|
|
@ -1883,7 +1883,7 @@ public class SolrCLI {
|
|||
log.warn("Skipping safety checks, configuration directory "+configName+" will be deleted with impunity.");
|
||||
} else {
|
||||
// need to scan all Collections to see if any are using the config
|
||||
Set<String> collections = zkStateReader.getClusterState().getCollections();
|
||||
Set<String> collections = zkStateReader.getClusterState().getCollectionsMap().keySet();
|
||||
|
||||
// give a little note to the user if there are many collections in case it takes a while
|
||||
if (collections.size() > 50)
|
||||
|
|
|
@ -66,21 +66,21 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("No collections found", 2, loadedClusterState.getCollections().size());
|
||||
assertEquals("Poperties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1"));
|
||||
assertEquals("Poperties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2"));
|
||||
assertEquals("No collections found", 2, loadedClusterState.getCollectionsMap().size());
|
||||
assertEquals("Properties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1"));
|
||||
assertEquals("Properties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2"));
|
||||
|
||||
loadedClusterState = ClusterState.load(-1, new byte[0], liveNodes);
|
||||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size());
|
||||
|
||||
loadedClusterState = ClusterState.load(-1, (byte[])null, liveNodes);
|
||||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size());
|
||||
}
|
||||
|
||||
public static ZkStateReader getMockZkStateReader(final Set<String> collections) {
|
||||
|
|
|
@ -1088,7 +1088,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
|||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("Could not find collection in :"
|
||||
+ clusterState.getCollections());
|
||||
+ clusterState.getCollectionsMap());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -525,7 +525,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
int maxIterations = 100;
|
||||
while (0 < maxIterations--) {
|
||||
final ClusterState state = stateReader.getClusterState();
|
||||
Set<String> availableCollections = state.getCollections();
|
||||
Set<String> availableCollections = state.getCollectionsMap().keySet();
|
||||
int availableCount = 0;
|
||||
for(String requiredCollection: collections) {
|
||||
if(availableCollections.contains(requiredCollection)) {
|
||||
|
@ -911,8 +911,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
ClusterState state = reader.getClusterState();
|
||||
|
||||
int numFound = 0;
|
||||
for (String c : state.getCollections()) {
|
||||
DocCollection collection = state.getCollection(c);
|
||||
Map<String, DocCollection> collectionsMap = state.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
DocCollection collection = entry.getValue();
|
||||
for (Slice slice : collection.getSlices()) {
|
||||
if (slice.getReplicasMap().get("core_node1") != null) {
|
||||
numFound++;
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete
|
|||
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||
|
@ -331,9 +332,10 @@ public class CloudSolrStream extends TupleStream implements Expressible {
|
|||
|
||||
if (slices == null) {
|
||||
//Try case insensitive match
|
||||
for(String col : clusterState.getCollections()) {
|
||||
if(col.equalsIgnoreCase(collection)) {
|
||||
slices = clusterState.getActiveSlices(col);
|
||||
Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
if (entry.getKey().equalsIgnoreCase(collection)) {
|
||||
slices = entry.getValue().getActiveSlices();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -465,9 +465,10 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
|
||||
if (slices == null) {
|
||||
//Try case insensitive match
|
||||
for(String col : clusterState.getCollections()) {
|
||||
if(col.equalsIgnoreCase(collection)) {
|
||||
slices = clusterState.getActiveSlices(col);
|
||||
Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
if (entry.getKey().equalsIgnoreCase(collection)) {
|
||||
slices = entry.getValue().getActiveSlices();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -188,6 +188,7 @@ public class UpdateStream extends TupleStream implements Expressible {
|
|||
child.setImplementingClass(getClass().getName());
|
||||
child.setExpressionType(ExpressionType.STREAM_DECORATOR);
|
||||
child.setExpression(toExpression(factory, false).toString());
|
||||
child.addChild(tupleSource.toExplanation(factory));
|
||||
|
||||
explanation.addChild(child);
|
||||
|
||||
|
|
|
@ -218,7 +218,10 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
* {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
|
||||
* because the semantics of how collection list is loaded have changed in SOLR-6629.
|
||||
* Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
|
||||
*
|
||||
* @deprecated use {@link #getCollectionsMap()} to avoid a second lookup for lazy collections
|
||||
*/
|
||||
@Deprecated
|
||||
public Set<String> getCollections() {
|
||||
Set<String> result = new HashSet<>();
|
||||
for (Entry<String, CollectionRef> entry : collectionStates.entrySet()) {
|
||||
|
@ -229,6 +232,27 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a map of collection name vs DocCollection objects
|
||||
*
|
||||
* Implementation note: This method resolves the collection reference by calling
|
||||
* {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
|
||||
* because the semantics of how collection list is loaded have changed in SOLR-6629.
|
||||
* Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
|
||||
*
|
||||
* @return a map of collection name vs DocCollection object
|
||||
*/
|
||||
public Map<String, DocCollection> getCollectionsMap() {
|
||||
Map<String, DocCollection> result = new HashMap<>(collectionStates.size());
|
||||
for (Entry<String, CollectionRef> entry : collectionStates.entrySet()) {
|
||||
DocCollection collection = entry.getValue().get();
|
||||
if (collection != null) {
|
||||
result.put(entry.getKey(), collection);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get names of the currently live nodes.
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -66,14 +67,14 @@ public class ClusterStateUtil {
|
|||
success = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections;
|
||||
Map<String, DocCollection> collections = null;
|
||||
if (collection != null) {
|
||||
collections = Collections.singleton(collection);
|
||||
collections = Collections.singletonMap(collection, clusterState.getCollection(collection));
|
||||
} else {
|
||||
collections = clusterState.getCollections();
|
||||
collections = clusterState.getCollectionsMap();
|
||||
}
|
||||
for (String coll : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(coll);
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
// only look at active shards
|
||||
|
@ -178,14 +179,14 @@ public class ClusterStateUtil {
|
|||
success = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections;
|
||||
if (collection == null) {
|
||||
collections = clusterState.getCollections();
|
||||
Map<String, DocCollection> collections = null;
|
||||
if (collection != null) {
|
||||
collections = Collections.singletonMap(collection, clusterState.getCollection(collection));
|
||||
} else {
|
||||
collections = Collections.singleton(collection);
|
||||
collections = clusterState.getCollectionsMap();
|
||||
}
|
||||
for (String coll : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(coll);
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
// only look at active shards
|
||||
|
|
|
@ -41,7 +41,8 @@ public class DocCollection extends ZkNodeProps {
|
|||
public static final String STATE_FORMAT = "stateFormat";
|
||||
public static final String RULE = "rule";
|
||||
public static final String SNITCH = "snitch";
|
||||
private int znodeVersion = -1; // sentinel
|
||||
|
||||
private final int znodeVersion;
|
||||
|
||||
private final String name;
|
||||
private final Map<String, Slice> slices;
|
||||
|
@ -55,7 +56,7 @@ public class DocCollection extends ZkNodeProps {
|
|||
|
||||
|
||||
public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
|
||||
this(name, slices, props, router, -1, ZkStateReader.CLUSTER_STATE);
|
||||
this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -64,8 +65,9 @@ public class DocCollection extends ZkNodeProps {
|
|||
* @param props The properties of the slice. This is used directly and a copy is not made.
|
||||
*/
|
||||
public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router, int zkVersion, String znode) {
|
||||
super(props==null ? props = new HashMap<String,Object>() : props);
|
||||
this.znodeVersion = zkVersion;
|
||||
super(props==null ? props = new HashMap<>() : props);
|
||||
// -1 means any version in ZK CAS, so we choose Integer.MAX_VALUE instead to avoid accidental overwrites
|
||||
this.znodeVersion = zkVersion == -1 ? Integer.MAX_VALUE : zkVersion;
|
||||
this.name = name;
|
||||
|
||||
this.slices = slices;
|
||||
|
|
|
@ -520,7 +520,8 @@ public class ZkStateReader implements Closeable {
|
|||
* In fact this is a clever way to avoid doing a ZK exists check on
|
||||
* the /collections/collection_name/state.json znode
|
||||
* Such an exists check is done in {@link ClusterState#hasCollection(String)} and
|
||||
* {@link ClusterState#getCollections()} method as a safeguard against exposing wrong collection names to the users
|
||||
* {@link ClusterState#getCollections()} and {@link ClusterState#getCollectionsMap()} methods
|
||||
* have a safeguard against exposing wrong collection names to the users
|
||||
*/
|
||||
private void refreshCollectionList(Watcher watcher) throws KeeperException, InterruptedException {
|
||||
synchronized (refreshCollectionListLock) {
|
||||
|
@ -715,8 +716,7 @@ public class ZkStateReader implements Closeable {
|
|||
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
|
||||
if (slices == null) {
|
||||
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
|
||||
"Could not find collection in zk: " + collection + " "
|
||||
+ clusterState.getCollections());
|
||||
"Could not find collection in zk: " + collection);
|
||||
}
|
||||
|
||||
Slice replicas = slices.get(shardId);
|
||||
|
|
|
@ -489,7 +489,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
List<String> collections = new ArrayList<>();
|
||||
collections.addAll(cluster.getSolrClient().getZkStateReader().getClusterState().getCollections());
|
||||
collections.addAll(cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionsMap().keySet());
|
||||
Collections.sort(collections);
|
||||
|
||||
try(ResultSet rs = databaseMetaData.getTables(null, zkHost, "%", null)) {
|
||||
|
|
|
@ -1353,7 +1353,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
|
||||
if (slices == null) {
|
||||
throw new RuntimeException("Could not find collection "
|
||||
+ DEFAULT_COLLECTION + " in " + clusterState.getCollections());
|
||||
+ DEFAULT_COLLECTION + " in " + clusterState.getCollectionsMap().keySet());
|
||||
}
|
||||
|
||||
for (CloudJettyRunner cjetty : cloudJettys) {
|
||||
|
@ -1916,9 +1916,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
if (collection != null) {
|
||||
cs = clusterState.getCollection(collection).toString();
|
||||
} else {
|
||||
Map<String,DocCollection> map = new HashMap<>();
|
||||
for (String coll : clusterState.getCollections())
|
||||
map.put(coll, clusterState.getCollection(coll));
|
||||
Map<String,DocCollection> map = clusterState.getCollectionsMap();
|
||||
CharArr out = new CharArr();
|
||||
new JSONWriter(out, 2).write(map);
|
||||
cs = out.toString();
|
||||
|
|
Loading…
Reference in New Issue