28 import astropy.units 
as u
 
   41 from lsst.verify 
import Job, Measurement
 
   44                                   ReferenceObjectLoader)
 
   46 from lsst.utils.timer 
import timeMethod
 
   48 from .dataIds 
import PerTractCcdDataIdContainer
 
   53 __all__ = [
"JointcalConfig", 
"JointcalRunner", 
"JointcalTask"]
 
   55 Photometry = collections.namedtuple(
'Photometry', (
'fit', 
'model'))
 
   56 Astrometry = collections.namedtuple(
'Astrometry', (
'fit', 
'model', 
'sky_to_tan_projection'))
 
   61     meas = Measurement(job.metrics[name], value)
 
   62     job.measurements.insert(meas)
 
   66     """Subclass of TaskRunner for jointcalTask (gen2) 
   68     jointcalTask.runDataRef() takes a number of arguments, one of which is a list of dataRefs 
   69     extracted from the command line (whereas most CmdLineTasks' runDataRef methods take 
   70     single dataRef, are are called repeatedly). This class transforms the processed 
   71     arguments generated by the ArgumentParser into the arguments expected by 
   72     Jointcal.runDataRef(). 
   74     See pipeBase.TaskRunner for more information. 
   80         Return a list of tuples per tract, each containing (dataRefs, kwargs). 
   82         Jointcal operates on lists of dataRefs simultaneously. 
   84         kwargs[
'butler'] = parsedCmd.butler
 
   88         for ref 
in parsedCmd.id.refList:
 
   89             refListDict.setdefault(ref.dataId[
"tract"], []).
append(ref)
 
   91         result = [(refListDict[tract], kwargs) 
for tract 
in sorted(refListDict.keys())]
 
   99             Arguments for Task.runDataRef() 
  104             if self.doReturnResults is False: 
  106             - ``exitStatus``: 0 if the task completed successfully, 1 otherwise. 
  108             if self.doReturnResults is True: 
  110             - ``result``: the result of calling jointcal.runDataRef() 
  111             - ``exitStatus``: 0 if the task completed successfully, 1 otherwise. 
  116         dataRefList, kwargs = args
 
  117         butler = kwargs.pop(
'butler')
 
  118         task = self.TaskClass(config=self.config, log=self.log, butler=butler)
 
  121             result = task.runDataRef(dataRefList, **kwargs)
 
  122             exitStatus = result.exitStatus
 
  123             job_path = butler.get(
'verify_job_filename')
 
  124             result.job.write(job_path[0])
 
  125         except Exception 
as e:  
 
  130                 eName = 
type(e).__name__
 
  131                 tract = dataRefList[0].dataId[
'tract']
 
  132                 task.log.fatal(
"Failed processing tract %s, %s: %s", tract, eName, e)
 
  135         kwargs[
'butler'] = butler
 
  136         if self.doReturnResults:
 
  137             return pipeBase.Struct(result=result, exitStatus=exitStatus)
 
  139             return pipeBase.Struct(exitStatus=exitStatus)
 
  143     """Lookup function that asserts/hopes that a static calibration dataset 
  144     exists in a particular collection, since this task can't provide a single 
  145     date/time to use to search for one properly. 
  147     This is mostly useful for the ``camera`` dataset, in cases where the task's 
  148     quantum dimensions do *not* include something temporal, like ``exposure`` 
  153     datasetType : `lsst.daf.butler.DatasetType` 
  154         Type of dataset being searched for. 
  155     registry : `lsst.daf.butler.Registry` 
  156         Data repository registry to search. 
  157     quantumDataId : `lsst.daf.butler.DataCoordinate` 
  158         Data ID of the quantum this camera should match. 
  159     collections : `Iterable` [ `str` ] 
  160         Collections that should be searched - but this lookup function works 
  161         by ignoring this in favor of a more-or-less hard-coded value. 
  165     refs : `Iterator` [ `lsst.daf.butler.DatasetRef` ] 
  166         Iterator over dataset references; should have only one element. 
  170     This implementation duplicates one in fgcmcal, and is at least quite 
  171     similar to another in cp_pipe.  This duplicate has the most documentation. 
  172     Fixing this is DM-29661. 
  174     instrument = Instrument.fromName(quantumDataId[
"instrument"], registry)
 
  175     unboundedCollection = instrument.makeUnboundedCalibrationRunName()
 
  176     return registry.queryDatasets(datasetType,
 
  177                                   dataId=quantumDataId,
 
  178                                   collections=[unboundedCollection],
 
  183     """Lookup function that finds all refcats for all visits that overlap a 
  184     tract, rather than just the refcats that directly overlap the tract. 
  188     datasetType : `lsst.daf.butler.DatasetType` 
  189         Type of dataset being searched for. 
  190     registry : `lsst.daf.butler.Registry` 
  191         Data repository registry to search. 
  192     quantumDataId : `lsst.daf.butler.DataCoordinate` 
  193         Data ID of the quantum; expected to be something we can use as a 
  194         constraint to query for overlapping visits. 
  195     collections : `Iterable` [ `str` ] 
  196         Collections to search. 
  200     refs : `Iterator` [ `lsst.daf.butler.DatasetRef` ] 
  201         Iterator over refcat references. 
  210     for visit_data_id 
in set(registry.queryDataIds(
"visit", dataId=quantumDataId).expanded()):
 
  212             registry.queryDatasets(
 
  214                 collections=collections,
 
  215                 dataId=visit_data_id,
 
  223                               dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter")):
 
  224     """Middleware input/output connections for jointcal data.""" 
  225     inputCamera = pipeBase.connectionTypes.PrerequisiteInput(
 
  226         doc=
"The camera instrument that took these observations.",
 
  228         storageClass=
"Camera",
 
  229         dimensions=(
"instrument",),
 
  231         lookupFunction=lookupStaticCalibrations,
 
  233     inputSourceTableVisit = pipeBase.connectionTypes.Input(
 
  234         doc=
"Source table in parquet format, per visit",
 
  235         name=
"sourceTable_visit",
 
  236         storageClass=
"DataFrame",
 
  237         dimensions=(
"instrument", 
"visit"),
 
  241     inputVisitSummary = pipeBase.connectionTypes.Input(
 
  242         doc=(
"Per-visit consolidated exposure metadata built from calexps. " 
  243              "These catalogs use detector id for the id and must be sorted for " 
  244              "fast lookups of a detector."),
 
  246         storageClass=
"ExposureCatalog",
 
  247         dimensions=(
"instrument", 
"visit"),
 
  251     astrometryRefCat = pipeBase.connectionTypes.PrerequisiteInput(
 
  252         doc=
"The astrometry reference catalog to match to loaded input catalog sources.",
 
  253         name=
"gaia_dr2_20200414",
 
  254         storageClass=
"SimpleCatalog",
 
  255         dimensions=(
"skypix",),
 
  258         lookupFunction=lookupVisitRefCats,
 
  260     photometryRefCat = pipeBase.connectionTypes.PrerequisiteInput(
 
  261         doc=
"The photometry reference catalog to match to loaded input catalog sources.",
 
  262         name=
"ps1_pv3_3pi_20170110",
 
  263         storageClass=
"SimpleCatalog",
 
  264         dimensions=(
"skypix",),
 
  267         lookupFunction=lookupVisitRefCats,
 
  270     outputWcs = pipeBase.connectionTypes.Output(
 
  271         doc=(
"Per-tract, per-visit world coordinate systems derived from the fitted model." 
  272              " These catalogs only contain entries for detectors with an output, and use" 
  273              " the detector id for the catalog id, sorted on id for fast lookups of a detector."),
 
  274         name=
"jointcalSkyWcsCatalog",
 
  275         storageClass=
"ExposureCatalog",
 
  276         dimensions=(
"instrument", 
"visit", 
"skymap", 
"tract"),
 
  279     outputPhotoCalib = pipeBase.connectionTypes.Output(
 
  280         doc=(
"Per-tract, per-visit photometric calibrations derived from the fitted model." 
  281              " These catalogs only contain entries for detectors with an output, and use" 
  282              " the detector id for the catalog id, sorted on id for fast lookups of a detector."),
 
  283         name=
"jointcalPhotoCalibCatalog",
 
  284         storageClass=
"ExposureCatalog",
 
  285         dimensions=(
"instrument", 
"visit", 
"skymap", 
"tract"),
 
  293     for name 
in (
"astrometry", 
"photometry"):
 
  294         vars()[f
"{name}_matched_fittedStars"] = pipeBase.connectionTypes.Output(
 
  295             doc=f
"The number of cross-matched fittedStars for {name}",
 
  296             name=f
"metricvalue_jointcal_{name}_matched_fittedStars",
 
  297             storageClass=
"MetricValue",
 
  298             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  300         vars()[f
"{name}_collected_refStars"] = pipeBase.connectionTypes.Output(
 
  301             doc=f
"The number of {name} reference stars drawn from the reference catalog, before matching.",
 
  302             name=f
"metricvalue_jointcal_{name}_collected_refStars",
 
  303             storageClass=
"MetricValue",
 
  304             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  306         vars()[f
"{name}_prepared_refStars"] = pipeBase.connectionTypes.Output(
 
  307             doc=f
"The number of {name} reference stars matched to fittedStars.",
 
  308             name=f
"metricvalue_jointcal_{name}_prepared_refStars",
 
  309             storageClass=
"MetricValue",
 
  310             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  312         vars()[f
"{name}_prepared_fittedStars"] = pipeBase.connectionTypes.Output(
 
  313             doc=f
"The number of cross-matched fittedStars after cleanup, for {name}.",
 
  314             name=f
"metricvalue_jointcal_{name}_prepared_fittedStars",
 
  315             storageClass=
"MetricValue",
 
  316             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  318         vars()[f
"{name}_prepared_ccdImages"] = pipeBase.connectionTypes.Output(
 
  319             doc=f
"The number of ccdImages that will be fit for {name}, after cleanup.",
 
  320             name=f
"metricvalue_jointcal_{name}_prepared_ccdImages",
 
  321             storageClass=
"MetricValue",
 
  322             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  324         vars()[f
"{name}_final_chi2"] = pipeBase.connectionTypes.Output(
 
  325             doc=f
"The final chi2 of the {name} fit.",
 
  326             name=f
"metricvalue_jointcal_{name}_final_chi2",
 
  327             storageClass=
"MetricValue",
 
  328             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  330         vars()[f
"{name}_final_ndof"] = pipeBase.connectionTypes.Output(
 
  331             doc=f
"The number of degrees of freedom of the fitted {name}.",
 
  332             name=f
"metricvalue_jointcal_{name}_final_ndof",
 
  333             storageClass=
"MetricValue",
 
  334             dimensions=(
"skymap", 
"tract", 
"instrument", 
"physical_filter"),
 
  345         if not config.doAstrometry:
 
  346             self.prerequisiteInputs.remove(
"astrometryRefCat")
 
  347             self.outputs.remove(
"outputWcs")
 
  348             for key 
in list(self.outputs):
 
  349                 if "metricvalue_jointcal_astrometry" in key:
 
  350                     self.outputs.remove(key)
 
  351         if not config.doPhotometry:
 
  352             self.prerequisiteInputs.remove(
"photometryRefCat")
 
  353             self.outputs.remove(
"outputPhotoCalib")
 
  354             for key 
in list(self.outputs):
 
  355                 if "metricvalue_jointcal_photometry" in key:
 
  356                     self.outputs.remove(key)
 
  360                      pipelineConnections=JointcalTaskConnections):
 
  361     """Configuration for JointcalTask""" 
  363     doAstrometry = pexConfig.Field(
 
  364         doc=
"Fit astrometry and write the fitted result.",
 
  368     doPhotometry = pexConfig.Field(
 
  369         doc=
"Fit photometry and write the fitted result.",
 
  373     coaddName = pexConfig.Field(
 
  374         doc=
"Type of coadd, typically deep or goodSeeing",
 
  379     sourceFluxType = pexConfig.Field(
 
  381         doc=
"Source flux field to use in source selection and to get fluxes from the catalog.",
 
  384     positionErrorPedestal = pexConfig.Field(
 
  385         doc=
"Systematic term to apply to the measured position error (pixels)",
 
  389     photometryErrorPedestal = pexConfig.Field(
 
  390         doc=
"Systematic term to apply to the measured error on flux or magnitude as a " 
  391         "fraction of source flux or magnitude delta (e.g. 0.05 is 5% of flux or +50 millimag).",
 
  396     matchCut = pexConfig.Field(
 
  397         doc=
"Matching radius between fitted and reference stars (arcseconds)",
 
  401     minMeasurements = pexConfig.Field(
 
  402         doc=
"Minimum number of associated measured stars for a fitted star to be included in the fit",
 
  406     minMeasuredStarsPerCcd = pexConfig.Field(
 
  407         doc=
"Minimum number of measuredStars per ccdImage before printing warnings",
 
  411     minRefStarsPerCcd = pexConfig.Field(
 
  412         doc=
"Minimum number of measuredStars per ccdImage before printing warnings",
 
  416     allowLineSearch = pexConfig.Field(
 
  417         doc=
"Allow a line search during minimization, if it is reasonable for the model" 
  418         " (models with a significant non-linear component, e.g. constrainedPhotometry).",
 
  422     astrometrySimpleOrder = pexConfig.Field(
 
  423         doc=
"Polynomial order for fitting the simple astrometry model.",
 
  427     astrometryChipOrder = pexConfig.Field(
 
  428         doc=
"Order of the per-chip transform for the constrained astrometry model.",
 
  432     astrometryVisitOrder = pexConfig.Field(
 
  433         doc=
"Order of the per-visit transform for the constrained astrometry model.",
 
  437     useInputWcs = pexConfig.Field(
 
  438         doc=
"Use the input calexp WCSs to initialize a SimpleAstrometryModel.",
 
  442     astrometryModel = pexConfig.ChoiceField(
 
  443         doc=
"Type of model to fit to astrometry",
 
  445         default=
"constrained",
 
  446         allowed={
"simple": 
"One polynomial per ccd",
 
  447                  "constrained": 
"One polynomial per ccd, and one polynomial per visit"}
 
  449     photometryModel = pexConfig.ChoiceField(
 
  450         doc=
"Type of model to fit to photometry",
 
  452         default=
"constrainedMagnitude",
 
  453         allowed={
"simpleFlux": 
"One constant zeropoint per ccd and visit, fitting in flux space.",
 
  454                  "constrainedFlux": 
"Constrained zeropoint per ccd, and one polynomial per visit," 
  455                  " fitting in flux space.",
 
  456                  "simpleMagnitude": 
"One constant zeropoint per ccd and visit," 
  457                  " fitting in magnitude space.",
 
  458                  "constrainedMagnitude": 
"Constrained zeropoint per ccd, and one polynomial per visit," 
  459                  " fitting in magnitude space.",
 
  462     applyColorTerms = pexConfig.Field(
 
  463         doc=
"Apply photometric color terms to reference stars?" 
  464             "Requires that colorterms be set to a ColortermLibrary",
 
  468     colorterms = pexConfig.ConfigField(
 
  469         doc=
"Library of photometric reference catalog name to color term dict.",
 
  470         dtype=ColortermLibrary,
 
  472     photometryVisitOrder = pexConfig.Field(
 
  473         doc=
"Order of the per-visit polynomial transform for the constrained photometry model.",
 
  477     photometryDoRankUpdate = pexConfig.Field(
 
  478         doc=(
"Do the rank update step during minimization. " 
  479              "Skipping this can help deal with models that are too non-linear."),
 
  483     astrometryDoRankUpdate = pexConfig.Field(
 
  484         doc=(
"Do the rank update step during minimization (should not change the astrometry fit). " 
  485              "Skipping this can help deal with models that are too non-linear."),
 
  489     outlierRejectSigma = pexConfig.Field(
 
  490         doc=
"How many sigma to reject outliers at during minimization.",
 
  494     astrometryOutlierRelativeTolerance = pexConfig.Field(
 
  495         doc=(
"Convergence tolerance for outlier rejection threshold when fitting astrometry. Iterations will " 
  496              "stop when the fractional change in the chi2 cut level is below this value. If tolerance is set " 
  497              "to zero, iterations will continue until there are no more outliers. We suggest a value of 0.002" 
  498              "as a balance between a shorter minimization runtime and achieving a final fitted model that is" 
  499              "close to the solution found when removing all outliers."),
 
  503     maxPhotometrySteps = pexConfig.Field(
 
  504         doc=
"Maximum number of minimize iterations to take when fitting photometry.",
 
  508     maxAstrometrySteps = pexConfig.Field(
 
  509         doc=
"Maximum number of minimize iterations to take when fitting astrometry.",
 
  513     astrometryRefObjLoader = pexConfig.ConfigurableField(
 
  514         target=LoadIndexedReferenceObjectsTask,
 
  515         doc=
"Reference object loader for astrometric fit",
 
  517     photometryRefObjLoader = pexConfig.ConfigurableField(
 
  518         target=LoadIndexedReferenceObjectsTask,
 
  519         doc=
"Reference object loader for photometric fit",
 
  521     sourceSelector = sourceSelectorRegistry.makeField(
 
  522         doc=
"How to select sources for cross-matching",
 
  525     astrometryReferenceSelector = pexConfig.ConfigurableField(
 
  526         target=ReferenceSourceSelectorTask,
 
  527         doc=
"How to down-select the loaded astrometry reference catalog.",
 
  529     photometryReferenceSelector = pexConfig.ConfigurableField(
 
  530         target=ReferenceSourceSelectorTask,
 
  531         doc=
"How to down-select the loaded photometry reference catalog.",
 
  533     astrometryReferenceErr = pexConfig.Field(
 
  534         doc=(
"Uncertainty on reference catalog coordinates [mas] to use in place of the `coord_*Err` fields. " 
  535              "If None, then raise an exception if the reference catalog is missing coordinate errors. " 
  536              "If specified, overrides any existing `coord_*Err` values."),
 
  543     writeInitMatrix = pexConfig.Field(
 
  545         doc=(
"Write the pre/post-initialization Hessian and gradient to text files, for debugging. " 
  546              "Output files will be written to `config.debugOutputPath` and will " 
  547              "be of the form 'astrometry_[pre|post]init-TRACT-FILTER-mat.txt'. " 
  548              "Note that these files are the dense versions of the matrix, and so may be very large."),
 
  551     writeChi2FilesInitialFinal = pexConfig.Field(
 
  553         doc=(
"Write .csv files containing the contributions to chi2 for the initialization and final fit. " 
  554              "Output files will be written to `config.debugOutputPath` and will " 
  555              "be of the form `astrometry_[initial|final]_chi2-TRACT-FILTER."),
 
  558     writeChi2FilesOuterLoop = pexConfig.Field(
 
  560         doc=(
"Write .csv files containing the contributions to chi2 for the outer fit loop. " 
  561              "Output files will be written to `config.debugOutputPath` and will " 
  562              "be of the form `astrometry_init-NN_chi2-TRACT-FILTER`."),
 
  565     writeInitialModel = pexConfig.Field(
 
  567         doc=(
"Write the pre-initialization model to text files, for debugging. " 
  568              "Output files will be written to `config.debugOutputPath` and will be " 
  569              "of the form `initial_astrometry_model-TRACT_FILTER.txt`."),
 
  572     debugOutputPath = pexConfig.Field(
 
  575         doc=(
"Path to write debug output files to. Used by " 
  576              "`writeInitialModel`, `writeChi2Files*`, `writeInitMatrix`.")
 
  578     detailedProfile = pexConfig.Field(
 
  581         doc=
"Output separate profiling information for different parts of jointcal, e.g. data read, fitting" 
  587             msg = 
"applyColorTerms=True requires the `colorterms` field be set to a ColortermLibrary." 
  588             raise pexConfig.FieldValidationError(JointcalConfig.colorterms, self, msg)
 
  590             msg = (
"Only doing astrometry, but Colorterms are not applied for astrometry;" 
  591                    "applyColorTerms=True will be ignored.")
 
  592             lsst.log.warning(msg)
 
  600         self.
sourceSelectorsourceSelector[
'science'].doSignalToNoise = 
True 
  603         self.
sourceSelectorsourceSelector[
'science'].signalToNoise.minimum = 10.
 
  605         fluxField = f
"slot_{self.sourceFluxType}Flux_instFlux" 
  606         self.
sourceSelectorsourceSelector[
'science'].signalToNoise.fluxField = fluxField
 
  607         self.
sourceSelectorsourceSelector[
'science'].signalToNoise.errField = fluxField + 
"Err" 
  613         badFlags = [
'base_PixelFlags_flag_edge', 
'base_PixelFlags_flag_saturated',
 
  614                     'base_PixelFlags_flag_interpolatedCenter', 
'base_SdssCentroid_flag',
 
  615                     'base_PsfFlux_flag', 
'base_PixelFlags_flag_suspectCenter']
 
  627     """Write model to outfile.""" 
  628     with open(filename, 
"w") 
as file:
 
  629         file.write(repr(model))
 
  630     log.info(
"Wrote %s to file: %s", model, filename)
 
  633 @dataclasses.dataclass 
  635     """The input data jointcal needs for each detector/visit.""" 
  637     """The visit identifier of this exposure.""" 
  639     """The catalog derived from this exposure.""" 
  641     """The VisitInfo of this exposure.""" 
  643     """The detector of this exposure.""" 
  645     """The photometric calibration of this exposure.""" 
  647     """The WCS of this exposure.""" 
  649     """The bounding box of this exposure.""" 
  651     """The filter of this exposure.""" 
  655     """Astrometricly and photometricly calibrate across multiple visits of the 
  660     butler : `lsst.daf.persistence.Butler` 
  661         The butler is passed to the refObjLoader constructor in case it is 
  662         needed. Ignored if the refObjLoader argument provides a loader directly. 
  663         Used to initialize the astrometry and photometry refObjLoaders. 
  664     initInputs : `dict`, optional 
  665         Dictionary used to initialize PipelineTasks (empty for jointcal). 
  668     ConfigClass = JointcalConfig
 
  669     RunnerClass = JointcalRunner
 
  670     _DefaultName = 
"jointcal" 
  672     def __init__(self, butler=None, initInputs=None, **kwargs):
 
  674         self.makeSubtask(
"sourceSelector")
 
  675         if self.config.doAstrometry:
 
  676             if initInputs 
is None:
 
  678                 self.makeSubtask(
'astrometryRefObjLoader', butler=butler)
 
  679             self.makeSubtask(
"astrometryReferenceSelector")
 
  682         if self.config.doPhotometry:
 
  683             if initInputs 
is None:
 
  685                 self.makeSubtask(
'photometryRefObjLoader', butler=butler)
 
  686             self.makeSubtask(
"photometryReferenceSelector")
 
  691         self.
jobjob = Job.load_metrics_package(subset=
'jointcal')
 
  696         inputs = butlerQC.get(inputRefs)
 
  698         tract = butlerQC.quantum.dataId[
'tract']
 
  699         if self.config.doAstrometry:
 
  701                 dataIds=[ref.datasetRef.dataId
 
  702                          for ref 
in inputRefs.astrometryRefCat],
 
  703                 refCats=inputs.pop(
'astrometryRefCat'),
 
  704                 config=self.config.astrometryRefObjLoader,
 
  706         if self.config.doPhotometry:
 
  708                 dataIds=[ref.datasetRef.dataId
 
  709                          for ref 
in inputRefs.photometryRefCat],
 
  710                 refCats=inputs.pop(
'photometryRefCat'),
 
  711                 config=self.config.photometryRefObjLoader,
 
  713         outputs = self.
runrun(**inputs, tract=tract)
 
  714         self.
_put_metrics_put_metrics(butlerQC, outputs.job, outputRefs)
 
  715         if self.config.doAstrometry:
 
  716             self.
_put_output_put_output(butlerQC, outputs.outputWcs, outputRefs.outputWcs,
 
  717                              inputs[
'inputCamera'], 
"setWcs")
 
  718         if self.config.doPhotometry:
 
  719             self.
_put_output_put_output(butlerQC, outputs.outputPhotoCalib, outputRefs.outputPhotoCalib,
 
  720                              inputs[
'inputCamera'], 
"setPhotoCalib")
 
  722     def _put_metrics(self, butlerQC, job, outputRefs):
 
  723         """Persist all measured metrics stored in a job. 
  727         butlerQC : `lsst.pipe.base.ButlerQuantumContext` 
  728             A butler which is specialized to operate in the context of a 
  729             `lsst.daf.butler.Quantum`; This is the input to `runQuantum`. 
  730         job : `lsst.verify.job.Job` 
  731             Measurements of metrics to persist. 
  732         outputRefs : `list` [`lsst.pipe.base.connectionTypes.OutputQuantizedConnection`] 
  733             The DatasetRefs to persist the data to. 
  735         for key 
in job.measurements.keys():
 
  736             butlerQC.put(job.measurements[key], getattr(outputRefs, key.fqn.replace(
'jointcal.', 
'')))
 
  738     def _put_output(self, butlerQC, outputs, outputRefs, camera, setter):
 
  739         """Persist the output datasets to their appropriate datarefs. 
  743         butlerQC : `lsst.pipe.base.ButlerQuantumContext` 
  744             A butler which is specialized to operate in the context of a 
  745             `lsst.daf.butler.Quantum`; This is the input to `runQuantum`. 
  746         outputs : `dict` [`tuple`, `lsst.afw.geom.SkyWcs`] or 
  747                   `dict` [`tuple, `lsst.afw.image.PhotoCalib`] 
  748             The fitted objects to persist. 
  749         outputRefs : `list` [`lsst.pipe.base.connectionTypes.OutputQuantizedConnection`] 
  750             The DatasetRefs to persist the data to. 
  751         camera : `lsst.afw.cameraGeom.Camera` 
  752             The camera for this instrument, to get detector ids from. 
  754             The method to call on the ExposureCatalog to set each output. 
  757         schema.addField(
'visit', type=
'I', doc=
'Visit number')
 
  759         def new_catalog(visit, size):
 
  760             """Return an catalog ready to be filled with appropriate output.""" 
  763             catalog[
'visit'] = visit
 
  765             metadata.add(
"COMMENT", 
"Catalog id is detector id, sorted.")
 
  766             metadata.add(
"COMMENT", 
"Only detectors with data have entries.")
 
  770         detectors_per_visit = collections.defaultdict(int)
 
  773             detectors_per_visit[key[0]] += 1
 
  775         for ref 
in outputRefs:
 
  776             visit = ref.dataId[
'visit']
 
  777             catalog = new_catalog(visit, detectors_per_visit[visit])
 
  780             for detector 
in camera:
 
  781                 detectorId = detector.getId()
 
  782                 key = (ref.dataId[
'visit'], detectorId)
 
  783                 if key 
not in outputs:
 
  785                     self.log.
debug(
"No %s output for detector %s in visit %s",
 
  786                                    setter[3:], detectorId, visit)
 
  789                 catalog[i].setId(detectorId)
 
  790                 getattr(catalog[i], setter)(outputs[key])
 
  794             butlerQC.put(catalog, ref)
 
  795             self.log.
info(
"Wrote %s detectors to %s", i, ref)
 
  797     def run(self, inputSourceTableVisit, inputVisitSummary, inputCamera, tract=None):
 
  803         sourceFluxField = 
"flux" 
  807         oldWcsList, bands = self.
_load_data_load_data(inputSourceTableVisit,
 
  813         boundingCircle, center, radius, defaultFilter, epoch = self.
_prep_sky_prep_sky(associations, bands)
 
  815         if self.config.doAstrometry:
 
  819                                                       referenceSelector=self.astrometryReferenceSelector,
 
  823             astrometry_output = self.
_make_output_make_output(associations.getCcdImageList(),
 
  827             astrometry_output = 
None 
  829         if self.config.doPhotometry:
 
  833                                                       referenceSelector=self.photometryReferenceSelector,
 
  837                                                       reject_bad_fluxes=
True)
 
  838             photometry_output = self.
_make_output_make_output(associations.getCcdImageList(),
 
  842             photometry_output = 
None 
  844         return pipeBase.Struct(outputWcs=astrometry_output,
 
  845                                outputPhotoCalib=photometry_output,
 
  850     def _make_schema_table(self):
 
  851         """Return an afw SourceTable to use as a base for creating the 
  852         SourceCatalog to insert values from the dataFrame into. 
  856         table : `lsst.afw.table.SourceTable` 
  857             Table with schema and slots to use to make SourceCatalogs. 
  860         schema.addField(
"centroid_x", 
"D")
 
  861         schema.addField(
"centroid_y", 
"D")
 
  862         schema.addField(
"centroid_xErr", 
"F")
 
  863         schema.addField(
"centroid_yErr", 
"F")
 
  864         schema.addField(
"shape_xx", 
"D")
 
  865         schema.addField(
"shape_yy", 
"D")
 
  866         schema.addField(
"shape_xy", 
"D")
 
  867         schema.addField(
"flux_instFlux", 
"D")
 
  868         schema.addField(
"flux_instFluxErr", 
"D")
 
  870         table.defineCentroid(
"centroid")
 
  871         table.defineShape(
"shape")
 
  874     def _extract_detector_catalog_from_visit_catalog(self, table, visitCatalog, detectorId,
 
  875                                                      detectorColumn, ixxColumns):
 
  876         """Return an afw SourceCatalog extracted from a visit-level dataframe, 
  877         limited to just one detector. 
  881         table : `lsst.afw.table.SourceTable` 
  882             Table factory to use to make the SourceCatalog that will be 
  883             populated with data from ``visitCatalog``. 
  884         visitCatalog : `pandas.DataFrame` 
  885             DataFrame to extract a detector catalog from. 
  887             Numeric id of the detector to extract from ``visitCatalog``. 
  888         detectorColumn : `str` 
  889             Name of the detector column in the catalog. 
  890         ixxColumns : `list` [`str`] 
  891             Names of the ixx/iyy/ixy columns in the catalog. 
  895         catalog : `lsst.afw.table.SourceCatalog` 
  896             Detector-level catalog extracted from ``visitCatalog``. 
  899         mapping = {
'x': 
'centroid_x',
 
  901                    'xErr': 
'centroid_xErr',
 
  902                    'yErr': 
'centroid_yErr',
 
  903                    ixxColumns[0]: 
'shape_xx',
 
  904                    ixxColumns[1]: 
'shape_yy',
 
  905                    ixxColumns[2]: 
'shape_xy',
 
  906                    f
'{self.config.sourceFluxType}_instFlux': 
'flux_instFlux',
 
  907                    f
'{self.config.sourceFluxType}_instFluxErr': 
'flux_instFluxErr',
 
  911         matched = visitCatalog[detectorColumn] == detectorId
 
  912         catalog.resize(sum(matched))
 
  913         view = visitCatalog.loc[matched]
 
  914         catalog[
'id'] = view.index
 
  915         for dfCol, afwCol 
in mapping.items():
 
  916             catalog[afwCol] = view[dfCol]
 
  918         self.log.
debug(
"%d sources selected in visit %d detector %d",
 
  920                        view[
'visit'].iloc[0],  
 
  924     def _load_data(self, inputSourceTableVisit, inputVisitSummary, associations,
 
  925                    jointcalControl, camera):
 
  926         """Read the data that jointcal needs to run. (Gen3 version) 
  928         Modifies ``associations`` in-place with the loaded data. 
  932         inputSourceTableVisit : `list` [`lsst.daf.butler.DeferredDatasetHandle`] 
  933             References to visit-level DataFrames to load the catalog data from. 
  934         inputVisitSummary : `list` [`lsst.daf.butler.DeferredDatasetHandle`] 
  935             Visit-level exposure summary catalog with metadata. 
  936         associations : `lsst.jointcal.Associations` 
  937             Object to add the loaded data to by constructing new CcdImages. 
  938         jointcalControl : `jointcal.JointcalControl` 
  939             Control object for C++ associations management. 
  940         camera : `lsst.afw.cameraGeom.Camera` 
  941             Camera object for detector geometry. 
  945         oldWcsList: `list` [`lsst.afw.geom.SkyWcs`] 
  946             The original WCS of the input data, to aid in writing tests. 
  947         bands : `list` [`str`] 
  948             The filter bands of each input dataset. 
  952         load_cat_prof_file = 
'jointcal_load_data.prof' if self.config.detailedProfile 
else '' 
  953         with pipeBase.cmdLineTask.profile(load_cat_prof_file):
 
  956             catalogMap = {ref.dataId[
'visit']: i 
for i, ref 
in enumerate(inputSourceTableVisit)}
 
  957             detectorDict = {detector.getId(): detector 
for detector 
in camera}
 
  961             for visitSummaryRef 
in inputVisitSummary:
 
  962                 visitSummary = visitSummaryRef.get()
 
  964                 dataRef = inputSourceTableVisit[catalogMap[visitSummaryRef.dataId[
'visit']]]
 
  966                     inColumns = dataRef.get(component=
'columns')
 
  968                 visitCatalog = dataRef.get(parameters={
'columns': columns})
 
  970                 selected = self.sourceSelector.
run(visitCatalog)
 
  973                 detectors = {id: index 
for index, id 
in enumerate(visitSummary[
'id'])}
 
  974                 for id, index 
in detectors.items():
 
  976                                                                                 detColumn, ixxColumns)
 
  977                     data = self.
_make_one_input_data_make_one_input_data(visitSummary[index], catalog, detectorDict)
 
  978                     result = self.
_build_ccdImage_build_ccdImage(data, associations, jointcalControl)
 
  979                     if result 
is not None:
 
  980                         oldWcsList.append(result.wcs)
 
  982                         filters.append(data.filter)
 
  983         if len(filters) == 0:
 
  984             raise RuntimeError(
"No data to process: did source selector remove all sources?")
 
  985         filters = collections.Counter(filters)
 
  987         return oldWcsList, filters
 
  989     def _make_one_input_data(self, visitRecord, catalog, detectorDict):
 
  990         """Return a data structure for this detector+visit.""" 
  993                                  visitInfo=visitRecord.getVisitInfo(),
 
  994                                  detector=detectorDict[visitRecord.getId()],
 
  995                                  photoCalib=visitRecord.getPhotoCalib(),
 
  996                                  wcs=visitRecord.getWcs(),
 
  997                                  bbox=visitRecord.getBBox(),
 
 1001                                                                    physical=visitRecord[
'physical_filter']))
 
 1003     def _get_sourceTable_visit_columns(self, inColumns):
 
 1005         Get the sourceTable_visit columns from the config. 
 1010             List of columns available in the sourceTable_visit 
 1015             List of columns to read from sourceTable_visit. 
 1016         detectorColumn : `str` 
 1017             Name of the detector column. 
 1019             Name of the ixx/iyy/ixy columns. 
 1021         if 'detector' in inColumns:
 
 1023             detectorColumn = 
'detector' 
 1026             detectorColumn = 
'ccd' 
 1028         columns = [
'visit', detectorColumn,
 
 1029                    'sourceId', 
'x', 
'xErr', 
'y', 
'yErr',
 
 1030                    self.config.sourceFluxType + 
'_instFlux', self.config.sourceFluxType + 
'_instFluxErr']
 
 1032         if 'ixx' in inColumns:
 
 1034             ixxColumns = [
'ixx', 
'iyy', 
'ixy']
 
 1037             ixxColumns = [
'Ixx', 
'Iyy', 
'Ixy']
 
 1038         columns.extend(ixxColumns)
 
 1040         if self.sourceSelector.config.doFlags:
 
 1041             columns.extend(self.sourceSelector.config.flags.bad)
 
 1042         if self.sourceSelector.config.doUnresolved:
 
 1043             columns.append(self.sourceSelector.config.unresolved.name)
 
 1044         if self.sourceSelector.config.doIsolated:
 
 1045             columns.append(self.sourceSelector.config.isolated.parentName)
 
 1046             columns.append(self.sourceSelector.config.isolated.nChildName)
 
 1048         return columns, detectorColumn, ixxColumns
 
 1053     def _getMetadataName(self):
 
 1057     def _makeArgumentParser(cls):
 
 1058         """Create an argument parser""" 
 1059         parser = pipeBase.ArgumentParser(name=cls.
_DefaultName_DefaultName)
 
 1060         parser.add_id_argument(
"--id", 
"calexp", help=
"data ID, e.g. --id visit=6789 ccd=0..9",
 
 1061                                ContainerClass=PerTractCcdDataIdContainer)
 
 1064     def _build_ccdImage(self, data, associations, jointcalControl):
 
 1066         Extract the necessary things from this catalog+metadata to add a new 
 1071         data : `JointcalInputData` 
 1072             The loaded input data. 
 1073         associations : `lsst.jointcal.Associations` 
 1074             Object to add the info to, to construct a new CcdImage 
 1075         jointcalControl : `jointcal.JointcalControl` 
 1076             Control object for associations management 
 1080         namedtuple or `None` 
 1082                 The TAN WCS of this image, read from the calexp 
 1083                 (`lsst.afw.geom.SkyWcs`). 
 1085                 A key to identify this dataRef by its visit and ccd ids 
 1088             if there are no sources in the loaded catalog. 
 1090         if len(data.catalog) == 0:
 
 1091             self.log.
warning(
"No sources selected in visit %s ccd %s", data.visit, data.detector.getId())
 
 1094         associations.createCcdImage(data.catalog,
 
 1098                                     data.filter.physicalLabel,
 
 1102                                     data.detector.getId(),
 
 1105         Result = collections.namedtuple(
'Result_from_build_CcdImage', (
'wcs', 
'key'))
 
 1106         Key = collections.namedtuple(
'Key', (
'visit', 
'ccd'))
 
 1107         return Result(data.wcs, Key(data.visit, data.detector.getId()))
 
 1109     def _readDataId(self, butler, dataId):
 
 1110         """Read all of the data for one dataId from the butler. (gen2 version)""" 
 1112         if "visit" in dataId.keys():
 
 1113             visit = dataId[
"visit"]
 
 1115             visit = butler.getButler().queryMetadata(
"calexp", (
"visit"), butler.dataId)[0]
 
 1116         detector = butler.get(
'calexp_detector', dataId=dataId)
 
 1118         catalog = butler.get(
'src',
 
 1119                              flags=lsst.afw.table.SOURCE_IO_NO_FOOTPRINTS,
 
 1121         goodSrc = self.sourceSelector.
run(catalog)
 
 1122         self.log.
debug(
"%d sources selected in visit %d detector %d",
 
 1123                        len(goodSrc.sourceCat),
 
 1127                                  catalog=goodSrc.sourceCat,
 
 1128                                  visitInfo=butler.get(
'calexp_visitInfo', dataId=dataId),
 
 1130                                  photoCalib=butler.get(
'calexp_photoCalib', dataId=dataId),
 
 1131                                  wcs=butler.get(
'calexp_wcs', dataId=dataId),
 
 1132                                  bbox=butler.get(
'calexp_bbox', dataId=dataId),
 
 1133                                  filter=butler.get(
'calexp_filterLabel', dataId=dataId))
 
 1135     def loadData(self, dataRefs, associations, jointcalControl):
 
 1136         """Read the data that jointcal needs to run. (Gen2 version)""" 
 1137         visit_ccd_to_dataRef = {}
 
 1140         load_cat_prof_file = 
'jointcal_loadData.prof' if self.config.detailedProfile 
else '' 
 1141         with pipeBase.cmdLineTask.profile(load_cat_prof_file):
 
 1143             camera = dataRefs[0].get(
'camera', immediate=
True)
 
 1145             for dataRef 
in dataRefs:
 
 1146                 data = self.
_readDataId_readDataId(dataRef.getButler(), dataRef.dataId)
 
 1147                 result = self.
_build_ccdImage_build_ccdImage(data, associations, jointcalControl)
 
 1150                 oldWcsList.append(result.wcs)
 
 1151                 visit_ccd_to_dataRef[result.key] = dataRef
 
 1152                 filters.append(data.filter)
 
 1153         if len(filters) == 0:
 
 1154             raise RuntimeError(
"No data to process: did source selector remove all sources?")
 
 1155         filters = collections.Counter(filters)
 
 1157         return oldWcsList, filters, visit_ccd_to_dataRef
 
 1159     def _getDebugPath(self, filename):
 
 1160         """Constructs a path to filename using the configured debug path. 
 1162         return os.path.join(self.config.debugOutputPath, filename)
 
 1164     def _prep_sky(self, associations, filters):
 
 1165         """Prepare on-sky and other data that must be computed after data has 
 1168         associations.computeCommonTangentPoint()
 
 1170         boundingCircle = associations.computeBoundingCircle()
 
 1172         radius = 
lsst.geom.Angle(boundingCircle.getOpeningAngle().asRadians(), lsst.geom.radians)
 
 1174         self.log.
info(f
"Data has center={center} with radius={radius.asDegrees()} degrees.")
 
 1177         defaultFilter = filters.most_common(1)[0][0]
 
 1178         self.log.
debug(
"Using '%s' filter for reference flux", defaultFilter.physicalLabel)
 
 1182         associations.setEpoch(epoch.jyear)
 
 1184         return boundingCircle, center, radius, defaultFilter, epoch
 
 1189         Jointly calibrate the astrometry and photometry across a set of images. 
 1191         NOTE: this is for gen2 middleware only. 
 1195         dataRefs : `list` of `lsst.daf.persistence.ButlerDataRef` 
 1196             List of data references to the exposures to be fit. 
 1200         result : `lsst.pipe.base.Struct` 
 1201             Struct of metadata from the fit, containing: 
 1204                 The provided data references that were fit (with updated WCSs) 
 1206                 The original WCS from each dataRef 
 1208                 Dictionary of internally-computed metrics for testing/validation. 
 1210         if len(dataRefs) == 0:
 
 1211             raise ValueError(
'Need a non-empty list of data references!')
 
 1215         sourceFluxField = 
"slot_%sFlux" % (self.config.sourceFluxType,)
 
 1219         oldWcsList, filters, visit_ccd_to_dataRef = self.
loadDataloadData(dataRefs,
 
 1223         boundingCircle, center, radius, defaultFilter, epoch = self.
_prep_sky_prep_sky(associations, filters)
 
 1225         tract = dataRefs[0].dataId[
'tract']
 
 1227         if self.config.doAstrometry:
 
 1231                                                       referenceSelector=self.astrometryReferenceSelector,
 
 1239         if self.config.doPhotometry:
 
 1243                                                       referenceSelector=self.photometryReferenceSelector,
 
 1247                                                       reject_bad_fluxes=
True)
 
 1252         return pipeBase.Struct(dataRefs=dataRefs,
 
 1253                                oldWcsList=oldWcsList,
 
 1257                                defaultFilter=defaultFilter,
 
 1259                                exitStatus=exitStatus)
 
 1261     def _get_refcat_coordinate_error_override(self, refCat, name):
 
 1262         """Check whether we should override the refcat coordinate errors, and 
 1263         return the overridden error if necessary. 
 1267         refCat : `lsst.afw.table.SimpleCatalog` 
 1268             The reference catalog to check for a ``coord_raErr`` field. 
 1270             Whether we are doing "astrometry" or "photometry". 
 1274         refCoordErr : `float` 
 1275             The refcat coordinate error to use, or NaN if we are not overriding 
 1280         lsst.pex.config.FieldValidationError 
 1281             Raised if the refcat does not contain coordinate errors and 
 1282             ``config.astrometryReferenceErr`` is not set. 
 1286         if name.lower() == 
"photometry":
 
 1287             if 'coord_raErr' not in refCat.schema:
 
 1292         if self.config.astrometryReferenceErr 
is None and 'coord_raErr' not in refCat.schema:
 
 1293             msg = (
"Reference catalog does not contain coordinate errors, " 
 1294                    "and config.astrometryReferenceErr not supplied.")
 
 1295             raise pexConfig.FieldValidationError(JointcalConfig.astrometryReferenceErr,
 
 1299         if self.config.astrometryReferenceErr 
is not None and 'coord_raErr' in refCat.schema:
 
 1300             self.log.
warning(
"Overriding reference catalog coordinate errors with %f/coordinate [mas]",
 
 1301                              self.config.astrometryReferenceErr)
 
 1303         if self.config.astrometryReferenceErr 
is None:
 
 1306             return self.config.astrometryReferenceErr
 
 1308     def _compute_proper_motion_epoch(self, ccdImageList):
 
 1309         """Return the proper motion correction epoch of the provided images. 
 1313         ccdImageList : `list` [`lsst.jointcal.CcdImage`] 
 1314             The images to compute the appropriate epoch for. 
 1318         epoch : `astropy.time.Time` 
 1319             The date to use for proper motion corrections. 
 1321         return astropy.time.Time(np.mean([ccdImage.getEpoch() 
for ccdImage 
in ccdImageList]),
 
 1325     def _do_load_refcat_and_fit(self, associations, defaultFilter, center, radius,
 
 1326                                 tract="", match_cut=3.0,
 
 1327                                 reject_bad_fluxes=False, *,
 
 1328                                 name="", refObjLoader=None, referenceSelector=None,
 
 1329                                 fit_function=None, epoch=None):
 
 1330         """Load reference catalog, perform the fit, and return the result. 
 1334         associations : `lsst.jointcal.Associations` 
 1335             The star/reference star associations to fit. 
 1336         defaultFilter : `lsst.afw.image.FilterLabel` 
 1337             filter to load from reference catalog. 
 1338         center : `lsst.geom.SpherePoint` 
 1339             ICRS center of field to load from reference catalog. 
 1340         radius : `lsst.geom.Angle` 
 1341             On-sky radius to load from reference catalog. 
 1343             Name of thing being fit: "astrometry" or "photometry". 
 1344         refObjLoader : `lsst.meas.algorithms.LoadReferenceObjectsTask` 
 1345             Reference object loader to use to load a reference catalog. 
 1346         referenceSelector : `lsst.meas.algorithms.ReferenceSourceSelectorTask` 
 1347             Selector to use to pick objects from the loaded reference catalog. 
 1348         fit_function : callable 
 1349             Function to call to perform fit (takes Associations object). 
 1350         tract : `str`, optional 
 1351             Name of tract currently being fit. 
 1352         match_cut : `float`, optional 
 1353             Radius in arcseconds to find cross-catalog matches to during 
 1354             associations.associateCatalogs. 
 1355         reject_bad_fluxes : `bool`, optional 
 1356             Reject refCat sources with NaN/inf flux or NaN/0 fluxErr. 
 1357         epoch : `astropy.time.Time`, optional 
 1358             Epoch to which to correct refcat proper motion and parallax, 
 1359             or `None` to not apply such corrections. 
 1363         result : `Photometry` or `Astrometry` 
 1364             Result of `fit_function()` 
 1366         self.log.
info(
"====== Now processing %s...", name)
 
 1369         associations.associateCatalogs(match_cut)
 
 1371                         associations.fittedStarListSize())
 
 1373         applyColorterms = 
False if name.lower() == 
"astrometry" else self.config.applyColorTerms
 
 1375                                                          center, radius, defaultFilter,
 
 1376                                                          applyColorterms=applyColorterms,
 
 1380         associations.collectRefStars(refCat,
 
 1381                                      self.config.matchCut*lsst.geom.arcseconds,
 
 1383                                      refCoordinateErr=refCoordErr,
 
 1384                                      rejectBadFluxes=reject_bad_fluxes)
 
 1386                         associations.refStarListSize())
 
 1388         associations.prepareFittedStars(self.config.minMeasurements)
 
 1392                         associations.nFittedStarsWithAssociatedRefStar())
 
 1394                         associations.fittedStarListSize())
 
 1396                         associations.nCcdImagesValidForFit())
 
 1398         load_cat_prof_file = 
'jointcal_fit_%s.prof'%name 
if self.config.detailedProfile 
else '' 
 1399         dataName = 
"{}_{}".
format(tract, defaultFilter.physicalLabel)
 
 1400         with pipeBase.cmdLineTask.profile(load_cat_prof_file):
 
 1401             result = fit_function(associations, dataName)
 
 1404         if self.config.writeChi2FilesInitialFinal:
 
 1405             baseName = self.
_getDebugPath_getDebugPath(f
"{name}_final_chi2-{dataName}")
 
 1406             result.fit.saveChi2Contributions(baseName+
"{type}")
 
 1407             self.log.
info(
"Wrote chi2 contributions files: %s", baseName)
 
 1411     def _load_reference_catalog(self, refObjLoader, referenceSelector, center, radius, filterLabel,
 
 1412                                 applyColorterms=False, epoch=None):
 
 1413         """Load the necessary reference catalog sources, convert fluxes to 
 1414         correct units, and apply color term corrections if requested. 
 1418         refObjLoader : `lsst.meas.algorithms.LoadReferenceObjectsTask` 
 1419             The reference catalog loader to use to get the data. 
 1420         referenceSelector : `lsst.meas.algorithms.ReferenceSourceSelectorTask` 
 1421             Source selector to apply to loaded reference catalog. 
 1422         center : `lsst.geom.SpherePoint` 
 1423             The center around which to load sources. 
 1424         radius : `lsst.geom.Angle` 
 1425             The radius around ``center`` to load sources in. 
 1426         filterLabel : `lsst.afw.image.FilterLabel` 
 1427             The camera filter to load fluxes for. 
 1428         applyColorterms : `bool` 
 1429             Apply colorterm corrections to the refcat for ``filterName``? 
 1430         epoch : `astropy.time.Time`, optional 
 1431             Epoch to which to correct refcat proper motion and parallax, 
 1432             or `None` to not apply such corrections. 
 1436         refCat : `lsst.afw.table.SimpleCatalog` 
 1437             The loaded reference catalog. 
 1439             The name of the reference catalog flux field appropriate for ``filterName``. 
 1441         skyCircle = refObjLoader.loadSkyCircle(center,
 
 1443                                                filterLabel.bandLabel,
 
 1446         selected = referenceSelector.run(skyCircle.refCat)
 
 1448         if not selected.sourceCat.isContiguous():
 
 1449             refCat = selected.sourceCat.copy(deep=
True)
 
 1451             refCat = selected.sourceCat
 
 1455                 refCatName = refObjLoader.ref_dataset_name  
 
 1456             except AttributeError:
 
 1457                 refCatName = self.config.connections.photometryRefCat  
 
 1459             self.log.
info(
"Applying color terms for physical filter=%r reference catalog=%s",
 
 1460                           filterLabel.physicalLabel, refCatName)
 
 1461             colorterm = self.config.colorterms.getColorterm(filterLabel.physicalLabel,
 
 1465             refMag, refMagErr = colorterm.getCorrectedMagnitudes(refCat)
 
 1466             refCat[skyCircle.fluxField] = u.Magnitude(refMag, u.ABmag).to_value(u.nJy)
 
 1470         return refCat, skyCircle.fluxField
 
 1472     def _check_star_lists(self, associations, name):
 
 1474         if associations.nCcdImagesValidForFit() == 0:
 
 1475             raise RuntimeError(
'No images in the ccdImageList!')
 
 1476         if associations.fittedStarListSize() == 0:
 
 1477             raise RuntimeError(
'No stars in the {} fittedStarList!'.
format(name))
 
 1478         if associations.refStarListSize() == 0:
 
 1479             raise RuntimeError(
'No stars in the {} reference star list!'.
format(name))
 
 1481     def _logChi2AndValidate(self, associations, fit, model, chi2Label, writeChi2Name=None):
 
 1482         """Compute chi2, log it, validate the model, and return chi2. 
 1486         associations : `lsst.jointcal.Associations` 
 1487             The star/reference star associations to fit. 
 1488         fit : `lsst.jointcal.FitterBase` 
 1489             The fitter to use for minimization. 
 1490         model : `lsst.jointcal.Model` 
 1491             The model being fit. 
 1493             Label to describe the chi2 (e.g. "Initialized", "Final"). 
 1494         writeChi2Name : `str`, optional 
 1495             Filename prefix to write the chi2 contributions to. 
 1496             Do not supply an extension: an appropriate one will be added. 
 1500         chi2: `lsst.jointcal.Chi2Accumulator` 
 1501             The chi2 object for the current fitter and model. 
 1506             Raised if chi2 is infinite or NaN. 
 1508             Raised if the model is not valid. 
 1510         if writeChi2Name 
is not None:
 
 1512             fit.saveChi2Contributions(fullpath+
"{type}")
 
 1513             self.log.
info(
"Wrote chi2 contributions files: %s", fullpath)
 
 1515         chi2 = fit.computeChi2()
 
 1516         self.log.
info(
"%s %s", chi2Label, chi2)
 
 1518         if not np.isfinite(chi2.chi2):
 
 1519             raise FloatingPointError(f
'{chi2Label} chi2 is invalid: {chi2}')
 
 1520         if not model.validate(associations.getCcdImageList(), chi2.ndof):
 
 1521             raise ValueError(
"Model is not valid: check log messages for warnings.")
 
 1524     def _fit_photometry(self, associations, dataName=None):
 
 1526         Fit the photometric data. 
 1530         associations : `lsst.jointcal.Associations` 
 1531             The star/reference star associations to fit. 
 1533             Name of the data being processed (e.g. "1234_HSC-Y"), for 
 1534             identifying debugging files. 
 1538         fit_result : `namedtuple` 
 1539             fit : `lsst.jointcal.PhotometryFit` 
 1540                 The photometric fitter used to perform the fit. 
 1541             model : `lsst.jointcal.PhotometryModel` 
 1542                 The photometric model that was fit. 
 1544         self.log.
info(
"=== Starting photometric fitting...")
 
 1547         if self.config.photometryModel == 
"constrainedFlux":
 
 1550                                                        visitOrder=self.config.photometryVisitOrder,
 
 1551                                                        errorPedestal=self.config.photometryErrorPedestal)
 
 1553             doLineSearch = self.config.allowLineSearch
 
 1554         elif self.config.photometryModel == 
"constrainedMagnitude":
 
 1557                                                             visitOrder=self.config.photometryVisitOrder,
 
 1558                                                             errorPedestal=self.config.photometryErrorPedestal)
 
 1560             doLineSearch = self.config.allowLineSearch
 
 1561         elif self.config.photometryModel == 
"simpleFlux":
 
 1563                                                   errorPedestal=self.config.photometryErrorPedestal)
 
 1564             doLineSearch = 
False   
 1565         elif self.config.photometryModel == 
"simpleMagnitude":
 
 1567                                                        errorPedestal=self.config.photometryErrorPedestal)
 
 1568             doLineSearch = 
False   
 1573         if self.config.writeChi2FilesInitialFinal:
 
 1574             baseName = f
"photometry_initial_chi2-{dataName}" 
 1577         if self.config.writeInitialModel:
 
 1578             fullpath = self.
_getDebugPath_getDebugPath(f
"initial_photometry_model-{dataName}.txt")
 
 1580         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialized", writeChi2Name=baseName)
 
 1582         def getChi2Name(whatToFit):
 
 1583             if self.config.writeChi2FilesOuterLoop:
 
 1584                 return f
"photometry_init-%s_chi2-{dataName}" % whatToFit
 
 1590         if self.config.writeInitMatrix:
 
 1591             dumpMatrixFile = self.
_getDebugPath_getDebugPath(f
"photometry_preinit-{dataName}")
 
 1594         if self.config.photometryModel.startswith(
"constrained"):
 
 1597             fit.minimize(
"ModelVisit", dumpMatrixFile=dumpMatrixFile)
 
 1598             self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize ModelVisit",
 
 1599                                      writeChi2Name=getChi2Name(
"ModelVisit"))
 
 1602         fit.minimize(
"Model", doLineSearch=doLineSearch, dumpMatrixFile=dumpMatrixFile)
 
 1604                                  writeChi2Name=getChi2Name(
"Model"))
 
 1606         fit.minimize(
"Fluxes")  
 
 1607         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize Fluxes",
 
 1608                                  writeChi2Name=getChi2Name(
"Fluxes"))
 
 1610         fit.minimize(
"Model Fluxes", doLineSearch=doLineSearch)
 
 1611         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize ModelFluxes",
 
 1612                                  writeChi2Name=getChi2Name(
"ModelFluxes"))
 
 1614         model.freezeErrorTransform()
 
 1615         self.log.
debug(
"Photometry error scales are frozen.")
 
 1619                                  self.config.maxPhotometrySteps,
 
 1622                                  doRankUpdate=self.config.photometryDoRankUpdate,
 
 1623                                  doLineSearch=doLineSearch,
 
 1630     def _fit_astrometry(self, associations, dataName=None):
 
 1632         Fit the astrometric data. 
 1636         associations : `lsst.jointcal.Associations` 
 1637             The star/reference star associations to fit. 
 1639             Name of the data being processed (e.g. "1234_HSC-Y"), for 
 1640             identifying debugging files. 
 1644         fit_result : `namedtuple` 
 1645             fit : `lsst.jointcal.AstrometryFit` 
 1646                 The astrometric fitter used to perform the fit. 
 1647             model : `lsst.jointcal.AstrometryModel` 
 1648                 The astrometric model that was fit. 
 1649             sky_to_tan_projection : `lsst.jointcal.ProjectionHandler` 
 1650                 The model for the sky to tangent plane projection that was used in the fit. 
 1653         self.log.
info(
"=== Starting astrometric fitting...")
 
 1655         associations.deprojectFittedStars()
 
 1662         if self.config.astrometryModel == 
"constrained":
 
 1664                                                              sky_to_tan_projection,
 
 1665                                                              chipOrder=self.config.astrometryChipOrder,
 
 1666                                                              visitOrder=self.config.astrometryVisitOrder)
 
 1667         elif self.config.astrometryModel == 
"simple":
 
 1669                                                         sky_to_tan_projection,
 
 1670                                                         self.config.useInputWcs,
 
 1672                                                         order=self.config.astrometrySimpleOrder)
 
 1677         if self.config.writeChi2FilesInitialFinal:
 
 1678             baseName = f
"astrometry_initial_chi2-{dataName}" 
 1681         if self.config.writeInitialModel:
 
 1682             fullpath = self.
_getDebugPath_getDebugPath(f
"initial_astrometry_model-{dataName}.txt")
 
 1684         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initial", writeChi2Name=baseName)
 
 1686         def getChi2Name(whatToFit):
 
 1687             if self.config.writeChi2FilesOuterLoop:
 
 1688                 return f
"astrometry_init-%s_chi2-{dataName}" % whatToFit
 
 1692         if self.config.writeInitMatrix:
 
 1693             dumpMatrixFile = self.
_getDebugPath_getDebugPath(f
"astrometry_preinit-{dataName}")
 
 1698         if self.config.astrometryModel == 
"constrained":
 
 1699             fit.minimize(
"DistortionsVisit", dumpMatrixFile=dumpMatrixFile)
 
 1700             self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize DistortionsVisit",
 
 1701                                      writeChi2Name=getChi2Name(
"DistortionsVisit"))
 
 1704         fit.minimize(
"Distortions", dumpMatrixFile=dumpMatrixFile)
 
 1705         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize Distortions",
 
 1706                                  writeChi2Name=getChi2Name(
"Distortions"))
 
 1708         fit.minimize(
"Positions")
 
 1709         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize Positions",
 
 1710                                  writeChi2Name=getChi2Name(
"Positions"))
 
 1712         fit.minimize(
"Distortions Positions")
 
 1713         self.
_logChi2AndValidate_logChi2AndValidate(associations, fit, model, 
"Initialize DistortionsPositions",
 
 1714                                  writeChi2Name=getChi2Name(
"DistortionsPositions"))
 
 1718                                  self.config.maxAstrometrySteps,
 
 1720                                  "Distortions Positions",
 
 1721                                  sigmaRelativeTolerance=self.config.astrometryOutlierRelativeTolerance,
 
 1722                                  doRankUpdate=self.config.astrometryDoRankUpdate,
 
 1728         return Astrometry(fit, model, sky_to_tan_projection)
 
 1730     def _check_stars(self, associations):
 
 1731         """Count measured and reference stars per ccd and warn/log them.""" 
 1732         for ccdImage 
in associations.getCcdImageList():
 
 1733             nMeasuredStars, nRefStars = ccdImage.countStars()
 
 1734             self.log.
debug(
"ccdImage %s has %s measured and %s reference stars",
 
 1735                            ccdImage.getName(), nMeasuredStars, nRefStars)
 
 1736             if nMeasuredStars < self.config.minMeasuredStarsPerCcd:
 
 1737                 self.log.
warning(
"ccdImage %s has only %s measuredStars (desired %s)",
 
 1738                                  ccdImage.getName(), nMeasuredStars, self.config.minMeasuredStarsPerCcd)
 
 1739             if nRefStars < self.config.minRefStarsPerCcd:
 
 1740                 self.log.
warning(
"ccdImage %s has only %s RefStars (desired %s)",
 
 1741                                  ccdImage.getName(), nRefStars, self.config.minRefStarsPerCcd)
 
 1743     def _iterate_fit(self, associations, fitter, max_steps, name, whatToFit,
 
 1745                      sigmaRelativeTolerance=0,
 
 1747                      doLineSearch=False):
 
 1748         """Run fitter.minimize up to max_steps times, returning the final chi2. 
 1752         associations : `lsst.jointcal.Associations` 
 1753             The star/reference star associations to fit. 
 1754         fitter : `lsst.jointcal.FitterBase` 
 1755             The fitter to use for minimization. 
 1757             Maximum number of steps to run outlier rejection before declaring 
 1758             convergence failure. 
 1759         name : {'photometry' or 'astrometry'} 
 1760             What type of data are we fitting (for logs and debugging files). 
 1762             Passed to ``fitter.minimize()`` to define the parameters to fit. 
 1763         dataName : `str`, optional 
 1764             Descriptive name for this dataset (e.g. tract and filter), 
 1766         sigmaRelativeTolerance : `float`, optional 
 1767             Convergence tolerance for the fractional change in the chi2 cut 
 1768             level for determining outliers. If set to zero, iterations will 
 1769             continue until there are no outliers. 
 1770         doRankUpdate : `bool`, optional 
 1771             Do an Eigen rank update during minimization, or recompute the full 
 1772             matrix and gradient? 
 1773         doLineSearch : `bool`, optional 
 1774             Do a line search for the optimum step during minimization? 
 1778         chi2: `lsst.jointcal.Chi2Statistic` 
 1779             The final chi2 after the fit converges, or is forced to end. 
 1784             Raised if the fitter fails with a non-finite value. 
 1786             Raised if the fitter fails for some other reason; 
 1787             log messages will provide further details. 
 1789         if self.config.writeInitMatrix:
 
 1790             dumpMatrixFile = self.
_getDebugPath_getDebugPath(f
"{name}_postinit-{dataName}")
 
 1794         oldChi2.chi2 = float(
"inf")
 
 1795         for i 
in range(max_steps):
 
 1796             if self.config.writeChi2FilesOuterLoop:
 
 1797                 writeChi2Name = f
"{name}_iterate_{i}_chi2-{dataName}" 
 1799                 writeChi2Name = 
None 
 1800             result = fitter.minimize(whatToFit,
 
 1801                                      self.config.outlierRejectSigma,
 
 1802                                      sigmaRelativeTolerance=sigmaRelativeTolerance,
 
 1803                                      doRankUpdate=doRankUpdate,
 
 1804                                      doLineSearch=doLineSearch,
 
 1805                                      dumpMatrixFile=dumpMatrixFile)
 
 1807             chi2 = self.
_logChi2AndValidate_logChi2AndValidate(associations, fitter, fitter.getModel(),
 
 1808                                             f
"Fit iteration {i}", writeChi2Name=writeChi2Name)
 
 1810             if result == MinimizeResult.Converged:
 
 1812                     self.log.
debug(
"fit has converged - no more outliers - redo minimization " 
 1813                                    "one more time in case we have lost accuracy in rank update.")
 
 1815                     result = fitter.minimize(whatToFit, self.config.outlierRejectSigma,
 
 1816                                              sigmaRelativeTolerance=sigmaRelativeTolerance)
 
 1817                     chi2 = self.
_logChi2AndValidate_logChi2AndValidate(associations, fitter, fitter.getModel(), 
"Fit completed")
 
 1820                 if chi2.chi2/chi2.ndof >= 4.0:
 
 1821                     self.log.
error(
"Potentially bad fit: High chi-squared/ndof.")
 
 1824             elif result == MinimizeResult.Chi2Increased:
 
 1825                 self.log.
warning(
"Still some outliers remaining but chi2 increased - retry")
 
 1827                 chi2Ratio = chi2.chi2 / oldChi2.chi2
 
 1829                     self.log.
warning(
'Significant chi2 increase by a factor of %.4g / %.4g = %.4g',
 
 1830                                      chi2.chi2, oldChi2.chi2, chi2Ratio)
 
 1837                     msg = (
"Large chi2 increase between steps: fit likely cannot converge." 
 1838                            " Try setting one or more of the `writeChi2*` config fields and looking" 
 1839                            " at how individual star chi2-values evolve during the fit.")
 
 1840                     raise RuntimeError(msg)
 
 1842             elif result == MinimizeResult.NonFinite:
 
 1843                 filename = self.
_getDebugPath_getDebugPath(
"{}_failure-nonfinite_chi2-{}.csv".
format(name, dataName))
 
 1845                 fitter.saveChi2Contributions(filename+
"{type}")
 
 1846                 msg = 
"Nonfinite value in chi2 minimization, cannot complete fit. Dumped star tables to: {}" 
 1847                 raise FloatingPointError(msg.format(filename))
 
 1848             elif result == MinimizeResult.Failed:
 
 1849                 raise RuntimeError(
"Chi2 minimization failure, cannot complete fit.")
 
 1851                 raise RuntimeError(
"Unxepected return code from minimize().")
 
 1853             self.log.
error(
"%s failed to converge after %d steps"%(name, max_steps))
 
 1857     def _make_output(self, ccdImageList, model, func):
 
 1858         """Return the internal jointcal models converted to the afw 
 1859         structures that will be saved to disk. 
 1863         ccdImageList : `lsst.jointcal.CcdImageList` 
 1864             The list of CcdImages to get the output for. 
 1865         model : `lsst.jointcal.AstrometryModel` or `lsst.jointcal.PhotometryModel` 
 1866             The internal jointcal model to convert for each `lsst.jointcal.CcdImage`. 
 1868             The name of the function to call on ``model`` to get the converted 
 1869             structure. Must accept an `lsst.jointcal.CcdImage`. 
 1873         output : `dict` [`tuple`, `lsst.jointcal.AstrometryModel`] or 
 1874                  `dict` [`tuple`, `lsst.jointcal.PhotometryModel`] 
 1875             The data to be saved, keyed on (visit, detector). 
 1878         for ccdImage 
in ccdImageList:
 
 1879             ccd = ccdImage.ccdId
 
 1880             visit = ccdImage.visit
 
 1881             self.log.
debug(
"%s for visit: %d, ccd: %d", func, visit, ccd)
 
 1882             output[(visit, ccd)] = getattr(model, func)(ccdImage)
 
 1885     def _write_astrometry_results(self, associations, model, visit_ccd_to_dataRef):
 
 1887         Write the fitted astrometric results to a new 'jointcal_wcs' dataRef. 
 1891         associations : `lsst.jointcal.Associations` 
 1892             The star/reference star associations to fit. 
 1893         model : `lsst.jointcal.AstrometryModel` 
 1894             The astrometric model that was fit. 
 1895         visit_ccd_to_dataRef : `dict` of Key: `lsst.daf.persistence.ButlerDataRef` 
 1896             Dict of ccdImage identifiers to dataRefs that were fit. 
 1898         ccdImageList = associations.getCcdImageList()
 
 1899         output = self.
_make_output_make_output(ccdImageList, model, 
"makeSkyWcs")
 
 1900         for key, skyWcs 
in output.items():
 
 1901             dataRef = visit_ccd_to_dataRef[key]
 
 1903                 dataRef.put(skyWcs, 
'jointcal_wcs')
 
 1905                 self.log.
fatal(
'Failed to write updated Wcs: %s', str(e))
 
 1908     def _write_photometry_results(self, associations, model, visit_ccd_to_dataRef):
 
 1910         Write the fitted photometric results to a new 'jointcal_photoCalib' dataRef. 
 1914         associations : `lsst.jointcal.Associations` 
 1915             The star/reference star associations to fit. 
 1916         model : `lsst.jointcal.PhotometryModel` 
 1917             The photoometric model that was fit. 
 1918         visit_ccd_to_dataRef : `dict` of Key: `lsst.daf.persistence.ButlerDataRef` 
 1919             Dict of ccdImage identifiers to dataRefs that were fit. 
 1922         ccdImageList = associations.getCcdImageList()
 
 1923         output = self.
_make_output_make_output(ccdImageList, model, 
"toPhotoCalib")
 
 1924         for key, photoCalib 
in output.items():
 
 1925             dataRef = visit_ccd_to_dataRef[key]
 
 1927                 dataRef.put(photoCalib, 
'jointcal_photoCalib')
 
 1929                 self.log.
fatal(
'Failed to write updated PhotoCalib: %s', str(e))
 
A representation of a detector in a mosaic camera.
 
A group of labels for a filter in an exposure or coadd.
 
The photometric calibration of an exposure.
 
Information about a single exposure of an imaging camera.
 
Custom catalog class for ExposureRecord/Table.
 
static Schema makeMinimalSchema()
Return a minimal schema for Exposure tables and records.
 
static std::shared_ptr< SourceTable > make(Schema const &schema, std::shared_ptr< IdFactory > const &idFactory)
Construct a new table.
 
static Schema makeMinimalSchema()
Return a minimal schema for Source tables and records.
 
Class for storing ordered metadata with comments.
 
A class representing an angle.
 
An integer coordinate rectangle.
 
Point in an unspecified spherical coordinate system.
 
The class that implements the relations between MeasuredStar and FittedStar.
 
Class that handles the astrometric least squares problem.
 
Simple structure to accumulate chi2 and ndof.
 
A multi-component model, fitting mappings for sensors and visits simultaneously.
 
A projection handler in which all CCDs from the same visit have the same tangent point.
 
Class that handles the photometric least squares problem.
 
A model where there is one independent transform per CcdImage.
 
def getTargetList(parsedCmd, **kwargs)
 
def __init__(self, *config=None)
 
def _load_data(self, inputSourceTableVisit, inputVisitSummary, associations, jointcalControl, camera)
 
def _compute_proper_motion_epoch(self, ccdImageList)
 
def _get_refcat_coordinate_error_override(self, refCat, name)
 
def _getDebugPath(self, filename)
 
def _check_star_lists(self, associations, name)
 
def _make_one_input_data(self, visitRecord, catalog, detectorDict)
 
def _iterate_fit(self, associations, fitter, max_steps, name, whatToFit, dataName="", sigmaRelativeTolerance=0, doRankUpdate=True, doLineSearch=False)
 
def _write_astrometry_results(self, associations, model, visit_ccd_to_dataRef)
 
def _check_stars(self, associations)
 
def _prep_sky(self, associations, filters)
 
def _readDataId(self, butler, dataId)
 
def runQuantum(self, butlerQC, inputRefs, outputRefs)
 
def _do_load_refcat_and_fit(self, associations, defaultFilter, center, radius, tract="", match_cut=3.0, reject_bad_fluxes=False, *name="", refObjLoader=None, referenceSelector=None, fit_function=None, epoch=None)
 
def _put_metrics(self, butlerQC, job, outputRefs)
 
def _build_ccdImage(self, data, associations, jointcalControl)
 
def runDataRef(self, dataRefs)
 
def __init__(self, butler=None, initInputs=None, **kwargs)
 
def _make_schema_table(self)
 
def _get_sourceTable_visit_columns(self, inColumns)
 
def run(self, inputSourceTableVisit, inputVisitSummary, inputCamera, tract=None)
 
def _fit_photometry(self, associations, dataName=None)
 
def _write_photometry_results(self, associations, model, visit_ccd_to_dataRef)
 
def _logChi2AndValidate(self, associations, fit, model, chi2Label, writeChi2Name=None)
 
def _extract_detector_catalog_from_visit_catalog(self, table, visitCatalog, detectorId, detectorColumn, ixxColumns)
 
def _make_output(self, ccdImageList, model, func)
 
def _put_output(self, butlerQC, outputs, outputRefs, camera, setter)
 
def _load_reference_catalog(self, refObjLoader, referenceSelector, center, radius, filterLabel, applyColorterms=False, epoch=None)
 
def _fit_astrometry(self, associations, dataName=None)
 
def loadData(self, dataRefs, associations, jointcalControl)
 
Provides consistent interface for LSST exceptions.
 
std::shared_ptr< FrameSet > append(FrameSet const &first, FrameSet const &second)
Construct a FrameSet that performs two transformations in series.
 
Backwards-compatibility support for depersisting the old Calib (FluxMag0/FluxMag0Err) objects.
 
double fluxErrFromABMagErr(double magErr, double mag) noexcept
Compute flux error in Janskys from AB magnitude error and AB magnitude.
 
def lookupStaticCalibrations(datasetType, registry, quantumDataId, collections)
 
def add_measurement(job, name, value)
 
def lookupVisitRefCats(datasetType, registry, quantumDataId, collections)
 
def writeModel(model, filename, log)
 
Fit spatial kernel using approximate fluxes for candidates, and solving a linear system of equations.
 
def format(config, name=None, writeSourceLine=True, prefix="", verbose=False)